diff --git a/.asf.yaml b/.asf.yaml index 5a7f5ff7221..5d88fe28742 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -29,16 +29,16 @@ notifications: # Read more here: https://github.com/apache/infrastructure-asfyaml github: collaborators: - - brandboat - - FrankYang0529 - - gongxuanzhang - m1a2st - - mingyen066 - - ShivsundarR - smjn - TaiJuWu - - xijiu + - brandboat - Yunyung + - xijiu + - chirag-wadhwa5 + - mingyen066 + - ShivsundarR + - Rancho-7 enabled_merge_buttons: squash: true squash_commit_message: PR_TITLE_AND_DESC diff --git a/.github/actions/run-gradle/action.yml b/.github/actions/run-gradle/action.yml index 8422defb273..9c8e0945184 100644 --- a/.github/actions/run-gradle/action.yml +++ b/.github/actions/run-gradle/action.yml @@ -83,6 +83,9 @@ runs: RUN_FLAKY_TESTS: ${{ inputs.run-flaky-tests }} TEST_XML_OUTPUT_DIR: ${{ inputs.test-xml-output }} TEST_VERBOSE: ${{ inputs.test-verbose }} + # This build step is invoked by build.yml to run junit tests only, + # Spotbugs is being run by that workflow via the "check" task and does not need to also be run here, + # since that is redundant. run: | set +e ./.github/scripts/thread-dump.sh & @@ -97,6 +100,8 @@ runs: -Pkafka.cluster.test.repeat=$TEST_REPEAT \ -Pkafka.test.verbose=$TEST_VERBOSE \ -PcommitId=xxxxxxxxxxxxxxxx \ + -x spotbugsMain \ + -x spotbugsTest \ $TEST_TASK exitcode="$?" echo "exitcode=$exitcode" >> $GITHUB_OUTPUT diff --git a/.github/workflows/build.yml b/.github/workflows/build.yml index fc5ee1bd755..5137f8bf372 100644 --- a/.github/workflows/build.yml +++ b/.github/workflows/build.yml @@ -127,7 +127,7 @@ jobs: - name: Setup Gradle uses: ./.github/actions/setup-gradle with: - java-version: 23 + java-version: 24 gradle-cache-read-only: ${{ !inputs.is-trunk }} gradle-cache-write-only: ${{ inputs.is-trunk }} develocity-access-key: ${{ secrets.DEVELOCITY_ACCESS_KEY }} @@ -181,7 +181,7 @@ jobs: fail-fast: false matrix: # If we change these, make sure to adjust ci-complete.yml - java: [ 23, 17 ] + java: [ 24, 17 ] run-flaky: [ true, false ] run-new: [ true, false ] exclude: @@ -270,7 +270,7 @@ jobs: python .github/scripts/junit.py \ --path build/junit-xml >> $GITHUB_STEP_SUMMARY - # This job downloads all the JUnit XML files and thread dumps from the JDK 23 test runs. + # This job downloads all the JUnit XML files and thread dumps from the JDK 24 test runs. # If any test job fails, we will not run this job. Also, if any thread dump artifacts # are present, this means there was a timeout in the tests and so we will not proceed # with catalog creation. @@ -288,7 +288,7 @@ jobs: - name: Download Thread Dumps uses: actions/download-artifact@v4 with: - pattern: junit-thread-dumps-23-* + pattern: junit-thread-dumps-24-* path: thread-dumps merge-multiple: true - name: Check For Thread Dump @@ -302,7 +302,7 @@ jobs: - name: Download JUnit XMLs uses: actions/download-artifact@v4 with: - pattern: junit-xml-23-* # Only look at JDK 23 tests for the test catalog + pattern: junit-xml-24-* # Only look at JDK 24 tests for the test catalog path: junit-xml merge-multiple: true - name: Collate Test Catalog diff --git a/.github/workflows/ci-complete.yml b/.github/workflows/ci-complete.yml index 8855c998df2..44d4f5a9c1d 100644 --- a/.github/workflows/ci-complete.yml +++ b/.github/workflows/ci-complete.yml @@ -44,7 +44,7 @@ jobs: fail-fast: false matrix: # Make sure these match build.yml - java: [ 23, 17 ] + java: [ 24, 17 ] run-flaky: [ true, false ] run-new: [ true, false ] exclude: diff --git a/README.md b/README.md index 4c2fc4e1f16..8b5fe4c332e 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ You need to have [Java](http://www.oracle.com/technetwork/java/javase/downloads/index.html) installed. -We build and test Apache Kafka with 17 and 23. The `release` parameter in javac is set to `11` for the clients +We build and test Apache Kafka with 17 and 24. The `release` parameter in javac is set to `11` for the clients and streams modules, and `17` for the rest, ensuring compatibility with their respective minimum Java versions. Similarly, the `release` parameter in scalac is set to `11` for the streams modules and `17` for the rest. diff --git a/build.gradle b/build.gradle index 5f9e81789cc..2b4f1294e9c 100644 --- a/build.gradle +++ b/build.gradle @@ -37,7 +37,7 @@ plugins { id 'org.nosphere.apache.rat' version "0.8.1" id "io.swagger.core.v3.swagger-gradle-plugin" version "${swaggerVersion}" - id "com.github.spotbugs" version '6.0.25' apply false + id "com.github.spotbugs" version '6.2.3' apply false id 'org.scoverage' version '8.0.3' apply false id 'com.gradleup.shadow' version '8.3.6' apply false id 'com.diffplug.spotless' version "6.25.0" diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java index 33309ffb63d..32663249e7a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImpl.java @@ -928,6 +928,9 @@ public class ShareConsumerImpl implements ShareConsumerDelegate { } private void stopFindCoordinatorOnClose() { + if (applicationEventHandler == null) { + return; + } log.debug("Stop finding coordinator during consumer close"); applicationEventHandler.add(new StopFindCoordinatorOnCloseEvent()); } @@ -944,6 +947,10 @@ public class ShareConsumerImpl implements ShareConsumerDelegate { * 2. leave the group */ private void sendAcknowledgementsAndLeaveGroup(final Timer timer, final AtomicReference firstException) { + if (applicationEventHandler == null || backgroundEventProcessor == null || + backgroundEventReaper == null || backgroundEventQueue == null) { + return; + } completeQuietly( () -> applicationEventHandler.addAndGet(new ShareAcknowledgeOnCloseEvent(acknowledgementsToSend(), calculateDeadlineMs(timer))), "Failed to send pending acknowledgements with a timeout(ms)=" + timer.timeoutMs(), firstException); @@ -1035,6 +1042,9 @@ public class ShareConsumerImpl implements ShareConsumerDelegate { * If the acknowledgement commit callback throws an exception, this method will throw an exception. */ private void handleCompletedAcknowledgements(boolean onClose) { + if (backgroundEventQueue == null || backgroundEventReaper == null || backgroundEventProcessor == null) { + return; + } // If the user gets any fatal errors, they will get these exceptions in the background queue. // While closing, we ignore these exceptions so that the consumers close successfully. processBackgroundEvents(onClose ? e -> (e instanceof GroupAuthorizationException diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index c4ace64b0e5..48be57f262b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -418,7 +418,7 @@ public class KafkaProducer implements Producer { this.transactionManager = configureTransactionState(config, logContext); // There is no need to do work required for adaptive partitioning, if we use a custom partitioner. boolean enableAdaptivePartitioning = partitionerPlugin.get() == null && - config.getBoolean(ProducerConfig.PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG); + config.getBoolean(ProducerConfig.PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG); RecordAccumulator.PartitionerConfig partitionerConfig = new RecordAccumulator.PartitionerConfig( enableAdaptivePartitioning, config.getLong(ProducerConfig.PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index e28b5d30812..ff7079bcdab 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -101,8 +101,10 @@ public class ProducerConfig extends AbstractConfig { + "similar or lower producer latency despite the increased linger."; /** partitioner.adaptive.partitioning.enable */ - public static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG = "partitioner.adaptive.partitioning.enable"; - private static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC = + public static final String PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG = "partitioner.adaptive.partitioning.enable"; + @Deprecated + public static final String PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG = PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG; + private static final String PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_DOC = "When set to 'true', the producer will try to adapt to broker performance and produce more messages to partitions hosted on faster brokers. " + "If 'false', the producer will try to distribute messages uniformly. Note: this setting has no effect if a custom partitioner is used."; @@ -111,7 +113,7 @@ public class ProducerConfig extends AbstractConfig { private static final String PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC = "If a broker cannot process produce requests from a partition for " + PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG + " time, " + "the partitioner treats that partition as not available. If the value is 0, this logic is disabled. " - + "Note: this setting has no effect if a custom partitioner is used or " + PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG + + "Note: this setting has no effect if a custom partitioner is used or " + PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG + " is set to 'false'."; /** partitioner.ignore.keys */ @@ -392,7 +394,7 @@ public class ProducerConfig extends AbstractConfig { .define(COMPRESSION_LZ4_LEVEL_CONFIG, Type.INT, CompressionType.LZ4.defaultLevel(), CompressionType.LZ4.levelValidator(), Importance.MEDIUM, COMPRESSION_LZ4_LEVEL_DOC) .define(COMPRESSION_ZSTD_LEVEL_CONFIG, Type.INT, CompressionType.ZSTD.defaultLevel(), CompressionType.ZSTD.levelValidator(), Importance.MEDIUM, COMPRESSION_ZSTD_LEVEL_DOC) .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC) - .define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC) + .define(PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADAPTIVE_PARTITIONING_ENABLE_DOC) .define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC) .define(PARTITIONER_IGNORE_KEYS_CONFIG, Type.BOOLEAN, false, Importance.MEDIUM, PARTITIONER_IGNORE_KEYS_DOC) .define(LINGER_MS_CONFIG, Type.LONG, 5, atLeast(0), Importance.MEDIUM, LINGER_MS_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 9f5a9a16a5e..958df8abc31 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -181,7 +181,8 @@ public class TopicConfig { "When used together, min.insync.replicas and acks allow you to enforce greater durability guarantees. " + "A typical scenario would be to create a topic with a replication factor of 3, " + "set min.insync.replicas to 2, and produce with acks of \"all\". " + - "This will ensure that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers."; + "This ensures that a majority of replicas must persist a write before it's considered successful by the producer and it's visible to consumers." + + "

Note that when the Eligible Leader Replicas feature is enabled, the semantics of this config changes. Please refer to the ELR section for more info.

"; public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. " + diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java index 6880ad16b8d..09fc99d8e24 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumerImplTest.java @@ -43,6 +43,7 @@ import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.LogCaptureAppender; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -77,6 +78,7 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -207,11 +209,19 @@ public class ShareConsumerImplTest { props.put(ConsumerConfig.GROUP_ID_CONFIG, "group-id"); props.put(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, "an.invalid.class"); final ConsumerConfig config = new ConsumerConfig(props); + + LogCaptureAppender appender = LogCaptureAppender.createAndRegister(); KafkaException ce = assertThrows( KafkaException.class, () -> newConsumer(config)); assertTrue(ce.getMessage().contains("Failed to construct Kafka share consumer"), "Unexpected exception message: " + ce.getMessage()); assertTrue(ce.getCause().getMessage().contains("Class an.invalid.class cannot be found"), "Unexpected cause: " + ce.getCause()); + + boolean npeLogged = appender.getEvents().stream() + .flatMap(event -> event.getThrowableInfo().stream()) + .anyMatch(str -> str.contains("NullPointerException")); + + assertFalse(npeLogged, "Unexpected NullPointerException during consumer construction"); } @Test diff --git a/config/broker.properties b/config/broker.properties index 61a536c9b30..4a75f0b12d6 100644 --- a/config/broker.properties +++ b/config/broker.properties @@ -75,8 +75,8 @@ log.dirs=/tmp/kraft-broker-logs num.partitions=1 # The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. -# This value is recommended to be increased for installations with data dirs located in RAID array. -num.recovery.threads.per.data.dir=1 +# This value is recommended to be increased based on the installation resources. +num.recovery.threads.per.data.dir=2 ############################# Internal Topic Settings ############################# # The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state" diff --git a/config/controller.properties b/config/controller.properties index 84963c95701..3cf3a58b606 100644 --- a/config/controller.properties +++ b/config/controller.properties @@ -75,8 +75,8 @@ log.dirs=/tmp/kraft-controller-logs num.partitions=1 # The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. -# This value is recommended to be increased for installations with data dirs located in RAID array. -num.recovery.threads.per.data.dir=1 +# This value is recommended to be increased based on the installation resources. +num.recovery.threads.per.data.dir=2 ############################# Internal Topic Settings ############################# # The replication factor for the group metadata internal topics "__consumer_offsets" and "__transaction_state" diff --git a/config/log4j2.yaml b/config/log4j2.yaml index 49bcf78d136..de263c57c92 100644 --- a/config/log4j2.yaml +++ b/config/log4j2.yaml @@ -44,7 +44,7 @@ Configuration: # State Change appender - name: StateChangeAppender fileName: "${sys:kafka.logs.dir}/state-change.log" - filePattern: "${sys:kafka.logs.dir}/stage-change.log.%d{yyyy-MM-dd-HH}" + filePattern: "${sys:kafka.logs.dir}/state-change.log.%d{yyyy-MM-dd-HH}" PatternLayout: pattern: "${logPattern}" TimeBasedTriggeringPolicy: diff --git a/config/server.properties b/config/server.properties index d4b1fe0bc4d..7f1773d354e 100644 --- a/config/server.properties +++ b/config/server.properties @@ -78,8 +78,8 @@ log.dirs=/tmp/kraft-combined-logs num.partitions=1 # The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. -# This value is recommended to be increased for installations with data dirs located in RAID array. -num.recovery.threads.per.data.dir=1 +# This value is recommended to be increased based on the installation resources. +num.recovery.threads.per.data.dir=2 ############################# Internal Topic Settings ############################# # The replication factor for the group metadata internal topics "__consumer_offsets", "__share_group_state" and "__transaction_state" diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index 8b3d60133a9..7fa5358f1c3 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -224,6 +224,7 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned { private JsonConverterConfig config; private Cache fromConnectSchemaCache; private Cache toConnectSchemaCache; + private Schema schema = null; // if a schema is provided in config, this schema will be used for all messages private final JsonSerializer serializer; private final JsonDeserializer deserializer; @@ -286,6 +287,16 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned { fromConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize())); toConnectSchemaCache = new SynchronizedCache<>(new LRUCache<>(config.schemaCacheSize())); + + try { + final byte[] schemaContent = config.schemaContent(); + if (schemaContent != null) { + final JsonNode schemaNode = deserializer.deserialize("", schemaContent); + this.schema = asConnectSchema(schemaNode); + } + } catch (SerializationException e) { + throw new DataException("Failed to parse schema in converter config due to serialization error: ", e); + } } @Override @@ -340,13 +351,16 @@ public class JsonConverter implements Converter, HeaderConverter, Versioned { throw new DataException("Converting byte[] to Kafka Connect data failed due to serialization error: ", e); } - if (config.schemasEnabled() && (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME))) - throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." + + if (config.schemasEnabled()) { + if (schema != null) { + return new SchemaAndValue(schema, convertToConnect(schema, jsonValue, config)); + } else if (!jsonValue.isObject() || jsonValue.size() != 2 || !jsonValue.has(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME) || !jsonValue.has(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME)) { + throw new DataException("JsonConverter with schemas.enable requires \"schema\" and \"payload\" fields and may not contain additional fields." + " If you are trying to deserialize plain JSON data, set schemas.enable=false in your converter configuration."); - - // The deserialized data should either be an envelope object containing the schema and the payload or the schema - // was stripped during serialization and we need to fill in an all-encompassing schema. - if (!config.schemasEnabled()) { + } + } else { + // The deserialized data should either be an envelope object containing the schema and the payload or the schema + // was stripped during serialization and we need to fill in an all-encompassing schema. ObjectNode envelope = JSON_NODE_FACTORY.objectNode(); envelope.set(JsonSchema.ENVELOPE_SCHEMA_FIELD_NAME, null); envelope.set(JsonSchema.ENVELOPE_PAYLOAD_FIELD_NAME, jsonValue); diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java index f02d54ac263..4d148250114 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverterConfig.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.connect.storage.ConverterConfig; +import java.nio.charset.StandardCharsets; import java.util.Locale; import java.util.Map; @@ -35,6 +36,11 @@ public final class JsonConverterConfig extends ConverterConfig { private static final String SCHEMAS_ENABLE_DOC = "Include schemas within each of the serialized values and keys."; private static final String SCHEMAS_ENABLE_DISPLAY = "Enable Schemas"; + public static final String SCHEMA_CONTENT_CONFIG = "schema.content"; + public static final String SCHEMA_CONTENT_DEFAULT = null; + private static final String SCHEMA_CONTENT_DOC = "When set, this is used as the schema for all messages. Otherwise, the schema will be included in the content of each message."; + private static final String SCHEMA_CONTENT_DISPLAY = "Schema Content"; + public static final String SCHEMAS_CACHE_SIZE_CONFIG = "schemas.cache.size"; public static final int SCHEMAS_CACHE_SIZE_DEFAULT = 1000; private static final String SCHEMAS_CACHE_SIZE_DOC = "The maximum number of schemas that can be cached in this converter instance."; @@ -61,6 +67,8 @@ public final class JsonConverterConfig extends ConverterConfig { orderInGroup++, Width.MEDIUM, SCHEMAS_ENABLE_DISPLAY); CONFIG.define(SCHEMAS_CACHE_SIZE_CONFIG, Type.INT, SCHEMAS_CACHE_SIZE_DEFAULT, Importance.HIGH, SCHEMAS_CACHE_SIZE_DOC, group, orderInGroup++, Width.MEDIUM, SCHEMAS_CACHE_SIZE_DISPLAY); + CONFIG.define(SCHEMA_CONTENT_CONFIG, Type.STRING, SCHEMA_CONTENT_DEFAULT, Importance.HIGH, SCHEMA_CONTENT_DOC, group, + orderInGroup++, Width.MEDIUM, SCHEMA_CONTENT_DISPLAY); group = "Serialization"; orderInGroup = 0; @@ -86,6 +94,7 @@ public final class JsonConverterConfig extends ConverterConfig { private final int schemaCacheSize; private final DecimalFormat decimalFormat; private final boolean replaceNullWithDefault; + private final byte[] schemaContent; public JsonConverterConfig(Map props) { super(CONFIG, props); @@ -93,6 +102,10 @@ public final class JsonConverterConfig extends ConverterConfig { this.schemaCacheSize = getInt(SCHEMAS_CACHE_SIZE_CONFIG); this.decimalFormat = DecimalFormat.valueOf(getString(DECIMAL_FORMAT_CONFIG).toUpperCase(Locale.ROOT)); this.replaceNullWithDefault = getBoolean(REPLACE_NULL_WITH_DEFAULT_CONFIG); + String schemaContentStr = getString(SCHEMA_CONTENT_CONFIG); + this.schemaContent = (schemaContentStr == null || schemaContentStr.isEmpty()) + ? null + : schemaContentStr.getBytes(StandardCharsets.UTF_8); } /** @@ -130,4 +143,15 @@ public final class JsonConverterConfig extends ConverterConfig { return replaceNullWithDefault; } + /** + * If a default schema is provided in the converter config, this will be + * used for all messages. + * + * This is only relevant if schemas are enabled. + * + * @return Schema Contents, will return null if no value is provided + */ + public byte[] schemaContent() { + return schemaContent; + } } diff --git a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java index 34010ddac05..200b33d1774 100644 --- a/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java +++ b/connect/json/src/test/java/org/apache/kafka/connect/json/JsonConverterTest.java @@ -36,6 +36,8 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; @@ -977,6 +979,58 @@ public class JsonConverterTest { assertEquals(AppInfoParser.getVersion(), converter.version()); } + @Test + public void testSchemaContentIsNull() { + Map config = new HashMap<>(); + config.put(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, null); + converter.configure(config, false); + byte[] jsonBytes = "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes(); + SchemaAndValue result = converter.toConnectData(TOPIC, jsonBytes); + assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), result); + } + + @Test + public void testSchemaContentIsEmptyString() { + converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, ""), false); + assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes())); + } + + @Test + public void testSchemaContentValidSchema() { + converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"string\" }"), false); + assertEquals(new SchemaAndValue(Schema.STRING_SCHEMA, "foo-bar-baz"), converter.toConnectData(TOPIC, "\"foo-bar-baz\"".getBytes())); + } + + @Test + public void testSchemaContentInValidSchema() { + assertThrows( + DataException.class, + () -> converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"string\" }"), false), + " Provided schema is invalid , please recheck the schema you have provided"); + } + + @Test + public void testSchemaContentLooksLikeSchema() { + converter.configure(Map.of(JsonConverterConfig.SCHEMA_CONTENT_CONFIG, "{ \"type\": \"struct\", \"fields\": [{\"field\": \"schema\", \"type\": \"struct\",\"fields\": [{\"field\": \"type\", \"type\": \"string\" }]}, {\"field\": \"payload\", \"type\": \"string\"}]}"), false); + SchemaAndValue connectData = converter.toConnectData(TOPIC, "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\" }".getBytes()); + assertEquals("foo-bar-baz", ((Struct) connectData.value()).getString("payload")); + } + + @ParameterizedTest + @ValueSource(strings = { + "{ }", + "{ \"wrong\": \"schema\" }", + "{ \"schema\": { \"type\": \"string\" } }", + "{ \"payload\": \"foo-bar-baz\" }", + "{ \"schema\": { \"type\": \"string\" }, \"payload\": \"foo-bar-baz\", \"extra\": \"field\" }", + }) + public void testNullSchemaContentWithWrongConnectDataValue(String value) { + converter.configure(Map.of(), false); + assertThrows( + DataException.class, + () -> converter.toConnectData(TOPIC, value.getBytes())); + } + private JsonNode parse(byte[] json) { try { return objectMapper.readTree(json); diff --git a/core/src/main/java/kafka/server/QuotaFactory.java b/core/src/main/java/kafka/server/QuotaFactory.java index 693decca6b4..b672be42650 100644 --- a/core/src/main/java/kafka/server/QuotaFactory.java +++ b/core/src/main/java/kafka/server/QuotaFactory.java @@ -51,61 +51,14 @@ public class QuotaFactory { } }; - public static class QuotaManagers { - private final ClientQuotaManager fetch; - private final ClientQuotaManager produce; - private final ClientRequestQuotaManager request; - private final ControllerMutationQuotaManager controllerMutation; - private final ReplicationQuotaManager leader; - private final ReplicationQuotaManager follower; - private final ReplicationQuotaManager alterLogDirs; - private final Optional> clientQuotaCallbackPlugin; - - public QuotaManagers(ClientQuotaManager fetch, ClientQuotaManager produce, ClientRequestQuotaManager request, - ControllerMutationQuotaManager controllerMutation, ReplicationQuotaManager leader, - ReplicationQuotaManager follower, ReplicationQuotaManager alterLogDirs, - Optional> clientQuotaCallbackPlugin) { - this.fetch = fetch; - this.produce = produce; - this.request = request; - this.controllerMutation = controllerMutation; - this.leader = leader; - this.follower = follower; - this.alterLogDirs = alterLogDirs; - this.clientQuotaCallbackPlugin = clientQuotaCallbackPlugin; - } - - public ClientQuotaManager fetch() { - return fetch; - } - - public ClientQuotaManager produce() { - return produce; - } - - public ClientRequestQuotaManager request() { - return request; - } - - public ControllerMutationQuotaManager controllerMutation() { - return controllerMutation; - } - - public ReplicationQuotaManager leader() { - return leader; - } - - public ReplicationQuotaManager follower() { - return follower; - } - - public ReplicationQuotaManager alterLogDirs() { - return alterLogDirs; - } - - public Optional> clientQuotaCallbackPlugin() { - return clientQuotaCallbackPlugin; - } + public record QuotaManagers(ClientQuotaManager fetch, + ClientQuotaManager produce, + ClientRequestQuotaManager request, + ControllerMutationQuotaManager controllerMutation, + ReplicationQuotaManager leader, + ReplicationQuotaManager follower, + ReplicationQuotaManager alterLogDirs, + Optional> clientQuotaCallbackPlugin) { public void shutdown() { fetch.shutdown(); diff --git a/core/src/main/java/kafka/server/share/DelayedShareFetch.java b/core/src/main/java/kafka/server/share/DelayedShareFetch.java index aa619ee91a8..969029a6ea5 100644 --- a/core/src/main/java/kafka/server/share/DelayedShareFetch.java +++ b/core/src/main/java/kafka/server/share/DelayedShareFetch.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.message.ShareFetchResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.server.LogReadResult; import org.apache.kafka.server.metrics.KafkaMetricsGroup; import org.apache.kafka.server.purgatory.DelayedOperation; @@ -368,6 +369,14 @@ public class DelayedShareFetch extends DelayedOperation { "topic partitions {}", shareFetch.groupId(), shareFetch.memberId(), sharePartitions.keySet()); } + // At this point, there could be delayed requests sitting in the purgatory which are waiting on + // DelayedShareFetchPartitionKeys corresponding to partitions, whose leader has been changed to a different broker. + // In that case, such partitions would not be able to get acquired, and the tryComplete will keep on returning false. + // Eventually the operation will get timed out and completed, but it might not get removed from the purgatory. + // This has been eventually left it like this because the purging mechanism will trigger whenever the number of completed + // but still being watched operations is larger than the purge interval. This purge interval is defined by the config + // share.fetch.purgatory.purge.interval.requests and is 1000 by default, thereby ensuring that such stale operations do not + // grow indefinitely. return false; } catch (Exception e) { log.error("Error processing delayed share fetch request", e); @@ -757,7 +766,8 @@ public class DelayedShareFetch extends DelayedOperation { * Case a: The partition is in an offline log directory on this broker * Case b: This broker does not know the partition it tries to fetch * Case c: This broker is no longer the leader of the partition it tries to fetch - * Case d: All remote storage read requests completed + * Case d: This broker is no longer the leader or follower of the partition it tries to fetch + * Case e: All remote storage read requests completed * @return boolean representing whether the remote fetch is completed or not. */ private boolean maybeCompletePendingRemoteFetch() { @@ -765,14 +775,20 @@ public class DelayedShareFetch extends DelayedOperation { for (TopicIdPartition topicIdPartition : pendingRemoteFetchesOpt.get().fetchOffsetMetadataMap().keySet()) { try { - replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); + Partition partition = replicaManager.getPartitionOrException(topicIdPartition.topicPartition()); + if (!partition.isLeader()) { + throw new NotLeaderException("Broker is no longer the leader of topicPartition: " + topicIdPartition); + } } catch (KafkaStorageException e) { // Case a log.debug("TopicPartition {} is in an offline log directory, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); canComplete = true; } catch (UnknownTopicOrPartitionException e) { // Case b log.debug("Broker no longer knows of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); canComplete = true; - } catch (NotLeaderOrFollowerException e) { // Case c + } catch (NotLeaderException e) { // Case c + log.debug("Broker is no longer the leader of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); + canComplete = true; + } catch (NotLeaderOrFollowerException e) { // Case d log.debug("Broker is no longer the leader or follower of topicPartition {}, satisfy {} immediately", topicIdPartition, shareFetch.fetchParams()); canComplete = true; } @@ -780,7 +796,7 @@ public class DelayedShareFetch extends DelayedOperation { break; } - if (canComplete || pendingRemoteFetchesOpt.get().isDone()) { // Case d + if (canComplete || pendingRemoteFetchesOpt.get().isDone()) { // Case e return forceComplete(); } else return false; diff --git a/core/src/main/java/kafka/server/share/SharePartition.java b/core/src/main/java/kafka/server/share/SharePartition.java index db9b862839c..68ced23a5f7 100644 --- a/core/src/main/java/kafka/server/share/SharePartition.java +++ b/core/src/main/java/kafka/server/share/SharePartition.java @@ -803,7 +803,7 @@ public class SharePartition { } InFlightState updateResult = inFlightBatch.tryUpdateBatchState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId); - if (updateResult == null) { + if (updateResult == null || updateResult.state() != RecordState.ACQUIRED) { log.info("Unable to acquire records for the batch: {} in share partition: {}-{}", inFlightBatch, groupId, topicIdPartition); continue; @@ -1009,12 +1009,7 @@ public class SharePartition { updatedStates.add(updateResult); stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount())); - - // If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state. - // This should not change the next fetch offset because the record is not available for acquisition - if (updateResult.state() != RecordState.ARCHIVED) { - updateFindNextFetchOffset(true); - } + // Do not update the next fetch offset as the offset has not completed the transition yet. } } return Optional.empty(); @@ -1054,12 +1049,7 @@ public class SharePartition { updatedStates.add(updateResult); stateBatches.add(new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount())); - - // If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state. - // This should not change the next fetch offset because the record is not available for acquisition - if (updateResult.state() != RecordState.ARCHIVED) { - updateFindNextFetchOffset(true); - } + // Do not update the next fetch offset as the batch has not completed the transition yet. } return Optional.empty(); } @@ -1641,7 +1631,7 @@ public class SharePartition { InFlightState updateResult = offsetState.getValue().tryUpdateState(RecordState.ACQUIRED, DeliveryCountOps.INCREASE, maxDeliveryCount, memberId); - if (updateResult == null) { + if (updateResult == null || updateResult.state() != RecordState.ACQUIRED) { log.trace("Unable to acquire records for the offset: {} in batch: {}" + " for the share partition: {}-{}", offsetState.getKey(), inFlightBatch, groupId, topicIdPartition); @@ -1941,12 +1931,7 @@ public class SharePartition { updatedStates.add(updateResult); stateBatches.add(new PersisterStateBatch(offsetState.getKey(), offsetState.getKey(), updateResult.state().id(), (short) updateResult.deliveryCount())); - // If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state. - // This should not change the next fetch offset because the record is not available for acquisition - if (recordState == RecordState.AVAILABLE - && updateResult.state() != RecordState.ARCHIVED) { - updateFindNextFetchOffset(true); - } + // Do not update the nextFetchOffset as the offset has not completed the transition yet. } } finally { lock.writeLock().unlock(); @@ -1996,13 +1981,7 @@ public class SharePartition { stateBatches.add( new PersisterStateBatch(inFlightBatch.firstOffset(), inFlightBatch.lastOffset(), updateResult.state().id(), (short) updateResult.deliveryCount())); - - // If the maxDeliveryCount limit has been exceeded, the record will be transitioned to ARCHIVED state. - // This should not change the nextFetchOffset because the record is not available for acquisition - if (recordState == RecordState.AVAILABLE - && updateResult.state() != RecordState.ARCHIVED) { - updateFindNextFetchOffset(true); - } + // Do not update the next fetch offset as the batch has not completed the transition yet. } finally { lock.writeLock().unlock(); } @@ -2445,22 +2424,22 @@ public class SharePartition { releaseAcquisitionLockOnTimeoutForPerOffsetBatch(inFlightBatch, stateBatches, memberId, firstOffset, lastOffset); } } - - if (!stateBatches.isEmpty()) { - writeShareGroupState(stateBatches).whenComplete((result, exception) -> { - if (exception != null) { - log.debug("Failed to write the share group state on acquisition lock timeout for share partition: {}-{} memberId: {}", - groupId, topicIdPartition, memberId, exception); - } - // Even if write share group state RPC call fails, we will still go ahead with the state transition. - // Update the cached state and start and end offsets after releasing the acquisition lock on timeout. - maybeUpdateCachedStateAndOffsets(); - }); - } } finally { lock.writeLock().unlock(); } + if (!stateBatches.isEmpty()) { + writeShareGroupState(stateBatches).whenComplete((result, exception) -> { + if (exception != null) { + log.debug("Failed to write the share group state on acquisition lock timeout for share partition: {}-{} memberId: {}", + groupId, topicIdPartition, memberId, exception); + } + // Even if write share group state RPC call fails, we will still go ahead with the state transition. + // Update the cached state and start and end offsets after releasing the acquisition lock on timeout. + maybeUpdateCachedStateAndOffsets(); + }); + } + // If we have an acquisition lock timeout for a share-partition, then we should check if // there is a pending share fetch request for the share-partition and complete it. // Skip null check for stateBatches, it should always be initialized if reached here. diff --git a/core/src/main/java/kafka/server/share/SharePartitionManager.java b/core/src/main/java/kafka/server/share/SharePartitionManager.java index 5f0bf1fa239..4cd7a61cf6b 100644 --- a/core/src/main/java/kafka/server/share/SharePartitionManager.java +++ b/core/src/main/java/kafka/server/share/SharePartitionManager.java @@ -774,6 +774,7 @@ public class SharePartitionManager implements AutoCloseable { if (sharePartition != null) { sharePartition.markFenced(); replicaManager.removeListener(sharePartitionKey.topicIdPartition().topicPartition(), sharePartition.listener()); + replicaManager.completeDelayedShareFetchRequest(new DelayedShareFetchGroupKey(sharePartitionKey.groupId(), sharePartitionKey.topicIdPartition())); } } diff --git a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala index 25b27302d3a..7b98c8c16fb 100644 --- a/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala +++ b/core/src/main/scala/kafka/server/AutoTopicCreationManager.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.message.CreateTopicsRequestData import org.apache.kafka.common.message.CreateTopicsRequestData.{CreatableTopic, CreatableTopicConfig, CreatableTopicConfigCollection} import org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.{CreateTopicsRequest, RequestContext, RequestHeader} +import org.apache.kafka.common.requests.{CreateTopicsRequest, CreateTopicsResponse, RequestContext, RequestHeader} import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator import org.apache.kafka.coordinator.transaction.TransactionLogConfig @@ -138,6 +138,19 @@ class DefaultAutoTopicCreationManager( } else if (response.versionMismatch() != null) { warn(s"Auto topic creation failed for ${creatableTopics.keys} with invalid version exception") } else { + if (response.hasResponse) { + response.responseBody() match { + case createTopicsResponse: CreateTopicsResponse => + createTopicsResponse.data().topics().forEach(topicResult => { + val error = Errors.forCode(topicResult.errorCode) + if (error != Errors.NONE) { + warn(s"Auto topic creation failed for ${topicResult.name} with error '${error.name}': ${topicResult.errorMessage}") + } + }) + case other => + warn(s"Auto topic creation request received unexpected response type: ${other.getClass.getSimpleName}") + } + } debug(s"Auto topic creation completed for ${creatableTopics.keys} with response ${response.responseBody}.") } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 67c6febe1ca..2bdadb02fb8 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -570,6 +570,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} (an explicit security mapping for each controller listener is required if ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} is non-empty, or if there are security protocols other than PLAINTEXT in use)") } } + // controller.quorum.auto.join.enable must be false for KRaft broker-only + require(!quorumConfig.autoJoin, + s"${QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG} is only supported when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role.") // warn that only the first controller listener is used if there is more than one if (controllerListenerNames.size > 1) { warn(s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} has multiple entries; only the first will be used since ${KRaftConfigs.PROCESS_ROLES_CONFIG}=broker: ${controllerListenerNames}") diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index 193c691308a..0703c5474f0 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -638,7 +638,7 @@ object DumpLogSegments { " Instead, the value-decoder-class option can be used if a custom RLMM implementation is configured.") private val shareStateOpt = parser.accepts("share-group-state-decoder", "If set, log data will be parsed as share group state data from the " + "__share_group_state topic.") - private val skipRecordMetadataOpt = parser.accepts("skip-record-metadata", "Whether to skip printing metadata for each record.") + private val skipRecordMetadataOpt = parser.accepts("skip-record-metadata", "Skip metadata when printing records. This flag also skips control records.") options = parser.parse(args : _*) def messageParser: MessageParser[_, _] = diff --git a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java index 5e21c6099e7..ad4193a0cb9 100644 --- a/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java +++ b/core/src/test/java/kafka/server/ReconfigurableQuorumIntegrationTest.java @@ -24,11 +24,14 @@ import org.apache.kafka.clients.admin.RaftVoterEndpoint; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.test.KafkaClusterTestKit; import org.apache.kafka.common.test.TestKitNodes; +import org.apache.kafka.common.test.api.TestKitDefaults; +import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.server.common.KRaftVersion; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; +import java.util.HashMap; import java.util.Map; import java.util.Set; import java.util.TreeMap; @@ -164,4 +167,69 @@ public class ReconfigurableQuorumIntegrationTest { } } } + + @Test + public void testControllersAutoJoinStandaloneVoter() throws Exception { + final var nodes = new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(3). + setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). + build(); + try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes). + setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true). + setStandalone(true). + build() + ) { + cluster.format(); + cluster.startup(); + try (Admin admin = Admin.create(cluster.clientProperties())) { + TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> { + Map voters = findVoterDirs(admin); + assertEquals(Set.of(3000, 3001, 3002), voters.keySet()); + for (int replicaId : new int[] {3000, 3001, 3002}) { + assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), voters.get(replicaId)); + } + }); + } + } + } + + @Test + public void testNewVoterAutoRemovesAndAdds() throws Exception { + final var nodes = new TestKitNodes.Builder(). + setNumBrokerNodes(1). + setNumControllerNodes(3). + setFeature(KRaftVersion.FEATURE_NAME, KRaftVersion.KRAFT_VERSION_1.featureLevel()). + build(); + + // Configure the initial voters with one voter having a different directory ID. + // This simulates the case where the controller failed and is brought back up with a different directory ID. + final Map initialVoters = new HashMap<>(); + final var oldDirectoryId = Uuid.randomUuid(); + for (final var controllerNode : nodes.controllerNodes().values()) { + initialVoters.put( + controllerNode.id(), + controllerNode.id() == TestKitDefaults.CONTROLLER_ID_OFFSET ? + oldDirectoryId : controllerNode.metadataDirectoryId() + ); + } + + try (KafkaClusterTestKit cluster = new KafkaClusterTestKit.Builder(nodes). + setConfigProp(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true). + setInitialVoterSet(initialVoters). + build() + ) { + cluster.format(); + cluster.startup(); + try (Admin admin = Admin.create(cluster.clientProperties())) { + TestUtils.retryOnExceptionWithTimeout(30_000, 10, () -> { + Map voters = findVoterDirs(admin); + assertEquals(Set.of(3000, 3001, 3002), voters.keySet()); + for (int replicaId : new int[] {3000, 3001, 3002}) { + assertEquals(nodes.controllerNodes().get(replicaId).metadataDirectoryId(), voters.get(replicaId)); + } + }); + } + } + } } diff --git a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java index 3b7701b7724..23f32659189 100644 --- a/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java +++ b/core/src/test/java/kafka/server/share/DelayedShareFetchTest.java @@ -153,6 +153,16 @@ public class DelayedShareFetchTest { when(sp0.canAcquireRecords()).thenReturn(false); when(sp1.canAcquireRecords()).thenReturn(false); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + ReplicaManager replicaManager = mock(ReplicaManager.class); + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(new MockTime()); Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() @@ -160,6 +170,7 @@ public class DelayedShareFetchTest { .withSharePartitions(sharePartitions) .withShareGroupMetrics(shareGroupMetrics) .withFetchId(fetchId) + .withReplicaManager(replicaManager) .build()); when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); @@ -218,6 +229,15 @@ public class DelayedShareFetchTest { PartitionMaxBytesStrategy partitionMaxBytesStrategy = mockPartitionMaxBytes(Set.of(tp0)); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + Time time = mock(Time.class); when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); @@ -287,6 +307,15 @@ public class DelayedShareFetchTest { mockTopicIdPartitionFetchBytes(replicaManager, tp0, hwmOffsetMetadata); BiConsumer exceptionHandler = mockExceptionHandler(); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) @@ -580,6 +609,19 @@ public class DelayedShareFetchTest { List delayedShareFetchWatchKeys = new ArrayList<>(); topicIdPartitions1.forEach(topicIdPartition -> delayedShareFetchWatchKeys.add(new DelayedShareFetchGroupKey(groupId, topicIdPartition.topicId(), topicIdPartition.partition()))); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + Partition p2 = mock(Partition.class); + when(p2.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2); + Uuid fetchId1 = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch1 = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch1) @@ -737,6 +779,12 @@ public class DelayedShareFetchTest { when(time.hiResClockMs()).thenReturn(100L).thenReturn(110L).thenReturn(170L); ShareGroupMetrics shareGroupMetrics = new ShareGroupMetrics(time); Uuid fetchId = Uuid.randomUuid(); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -881,10 +929,18 @@ public class DelayedShareFetchTest { BROKER_TOPIC_STATS); Uuid fetchId = Uuid.randomUuid(); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + ReplicaManager replicaManager = mock(ReplicaManager.class); + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + DelayedShareFetch delayedShareFetch = DelayedShareFetchTest.DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) .withFetchId(fetchId) + .withReplicaManager(replicaManager) .build(); when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); @@ -1263,6 +1319,19 @@ public class DelayedShareFetchTest { when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class)); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + Partition p2 = mock(Partition.class); + when(p2.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) @@ -1288,6 +1357,70 @@ public class DelayedShareFetchTest { delayedShareFetch.lock().unlock(); } + @Test + public void testRemoteStorageFetchPartitionLeaderChanged() { + ReplicaManager replicaManager = mock(ReplicaManager.class); + TopicIdPartition tp0 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0)); + + SharePartition sp0 = mock(SharePartition.class); + + when(sp0.canAcquireRecords()).thenReturn(true); + + LinkedHashMap sharePartitions = new LinkedHashMap<>(); + sharePartitions.put(tp0, sp0); + + ShareFetch shareFetch = new ShareFetch(FETCH_PARAMS, "grp", Uuid.randomUuid().toString(), + new CompletableFuture<>(), List.of(tp0), BATCH_SIZE, MAX_FETCH_RECORDS, + BROKER_TOPIC_STATS); + + when(sp0.nextFetchOffset()).thenReturn(10L); + + // Fetch offset does not match with the cached entry for sp0, hence, a replica manager fetch will happen for sp0. + when(sp0.fetchOffsetMetadata(anyLong())).thenReturn(Optional.empty()); + + // Mocking remote storage read result for tp0. + doAnswer(invocation -> buildLocalAndRemoteFetchResult(Set.of(), Set.of(tp0))).when(replicaManager).readFromLog(any(), any(), any(ReplicaQuota.class), anyBoolean()); + + // Remote fetch related mocks. Remote fetch object does not complete within tryComplete in this mock. + RemoteLogManager remoteLogManager = mock(RemoteLogManager.class); + when(remoteLogManager.asyncRead(any(), any())).thenReturn(mock(Future.class)); + when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(false); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + + Uuid fetchId = Uuid.randomUuid(); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() + .withShareFetchData(shareFetch) + .withSharePartitions(sharePartitions) + .withReplicaManager(replicaManager) + .withPartitionMaxBytesStrategy(mockPartitionMaxBytes(Set.of(tp0))) + .withFetchId(fetchId) + .build()); + + // All the topic partitions are acquirable. + when(sp0.maybeAcquireFetchLock(fetchId)).thenReturn(true); + + // Mock the behaviour of replica manager such that remote storage fetch completion timer task completes on adding it to the watch queue. + doAnswer(invocationOnMock -> { + TimerTask timerTask = invocationOnMock.getArgument(0); + timerTask.run(); + return null; + }).when(replicaManager).addShareFetchTimerRequest(any()); + + assertFalse(delayedShareFetch.isCompleted()); + assertTrue(delayedShareFetch.tryComplete()); + assertTrue(delayedShareFetch.isCompleted()); + // Remote fetch object gets created for delayed share fetch object. + assertNotNull(delayedShareFetch.pendingRemoteFetches()); + // Verify the locks are released for local log read topic partitions tp0. + Mockito.verify(delayedShareFetch, times(1)).releasePartitionLocks(Set.of(tp0)); + assertTrue(delayedShareFetch.lock().tryLock()); + delayedShareFetch.lock().unlock(); + } + @Test public void testRemoteStorageFetchTryCompleteThrowsException() { ReplicaManager replicaManager = mock(ReplicaManager.class); @@ -1516,6 +1649,16 @@ public class DelayedShareFetchTest { when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); Uuid fetchId = Uuid.randomUuid(); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -1586,6 +1729,12 @@ public class DelayedShareFetchTest { when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); Uuid fetchId = Uuid.randomUuid(); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) @@ -1679,6 +1828,19 @@ public class DelayedShareFetchTest { }).when(remoteLogManager).asyncRead(any(), any()); when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + Partition p2 = mock(Partition.class); + when(p2.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + when(replicaManager.getPartitionOrException(tp2.topicPartition())).thenReturn(p2); + Uuid fetchId = Uuid.randomUuid(); DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) @@ -1761,6 +1923,16 @@ public class DelayedShareFetchTest { when(replicaManager.remoteLogManager()).thenReturn(Option.apply(remoteLogManager)); Uuid fetchId = Uuid.randomUuid(); + + Partition p0 = mock(Partition.class); + when(p0.isLeader()).thenReturn(true); + + Partition p1 = mock(Partition.class); + when(p1.isLeader()).thenReturn(true); + + when(replicaManager.getPartitionOrException(tp0.topicPartition())).thenReturn(p0); + when(replicaManager.getPartitionOrException(tp1.topicPartition())).thenReturn(p1); + DelayedShareFetch delayedShareFetch = spy(DelayedShareFetchBuilder.builder() .withShareFetchData(shareFetch) .withSharePartitions(sharePartitions) diff --git a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java index 5d2d2e9a377..399a8dee761 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionManagerTest.java @@ -2622,7 +2622,8 @@ public class SharePartitionManagerTest { assertEquals(Errors.FENCED_STATE_EPOCH.code(), partitionDataMap.get(tp2).errorCode()); assertEquals("Fenced state epoch", partitionDataMap.get(tp2).errorMessage()); - Mockito.verify(replicaManager, times(0)).completeDelayedShareFetchRequest(any()); + Mockito.verify(replicaManager, times(1)).completeDelayedShareFetchRequest( + new DelayedShareFetchGroupKey(groupId, tp2)); Mockito.verify(replicaManager, times(1)).readFromLog( any(), any(), any(ReplicaQuota.class), anyBoolean()); // Should have 1 fetch recorded and 1 failure as single topic has multiple partition fetch diff --git a/core/src/test/java/kafka/server/share/SharePartitionTest.java b/core/src/test/java/kafka/server/share/SharePartitionTest.java index 1289d720054..30b49e17b16 100644 --- a/core/src/test/java/kafka/server/share/SharePartitionTest.java +++ b/core/src/test/java/kafka/server/share/SharePartitionTest.java @@ -7878,6 +7878,105 @@ public class SharePartitionTest { assertNull(sharePartition.cachedState().get(0L).batchAcquisitionLockTimeoutTask()); } + @Test + public void testRecordArchivedWithWriteStateRPCFailure() throws InterruptedException { + Persister persister = Mockito.mock(Persister.class); + SharePartition sharePartition = SharePartitionBuilder.builder() + .withState(SharePartitionState.ACTIVE) + .withDefaultAcquisitionLockTimeoutMs(ACQUISITION_LOCK_TIMEOUT_MS) + .withMaxDeliveryCount(2) + .withPersister(persister) + .build(); + + fetchAcquiredRecords(sharePartition, memoryRecords(5, 2), 5); + fetchAcquiredRecords(sharePartition, memoryRecords(5, 7), 5); + + // Futures which will be completed later, so the batch state has ongoing transition. + CompletableFuture future1 = new CompletableFuture<>(); + CompletableFuture future2 = new CompletableFuture<>(); + Mockito.when(persister.writeState(Mockito.any())).thenReturn(future1).thenReturn(future2); + + // Acknowledge batches. + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(3, 3, List.of(AcknowledgeType.ACCEPT.id)))); + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(7, 11, List.of(AcknowledgeType.ACCEPT.id)))); + + assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(2L).offsetState().get(3L).state()); + assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount()); + assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); + + WriteShareGroupStateResult writeShareGroupStateResult = Mockito.mock(WriteShareGroupStateResult.class); + Mockito.when(writeShareGroupStateResult.topicsData()).thenReturn(List.of( + new TopicData<>(TOPIC_ID_PARTITION.topicId(), List.of( + PartitionFactory.newPartitionErrorData(0, Errors.GROUP_ID_NOT_FOUND.code(), Errors.GROUP_ID_NOT_FOUND.message()))))); + + future1.complete(writeShareGroupStateResult); + assertEquals(12, sharePartition.nextFetchOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).offsetState().get(3L).state()); + assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount()); + assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); + + future2.complete(writeShareGroupStateResult); + assertEquals(12L, sharePartition.nextFetchOffset()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(2L).offsetState().get(3L).state()); + assertEquals(1, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount()); + assertEquals(RecordState.ACQUIRED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(1, sharePartition.cachedState().get(7L).batchDeliveryCount()); + + // Allowing acquisition lock to expire. This will also ensure that acquisition lock timeout task + // is run successfully post write state RPC failure. + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); + TestUtils.waitForCondition( + () -> sharePartition.cachedState().get(2L).offsetState().get(3L).state() == RecordState.AVAILABLE && + sharePartition.cachedState().get(7L).batchState() == RecordState.AVAILABLE && + sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount() == 1 && + sharePartition.cachedState().get(7L).batchDeliveryCount() == 1 && + sharePartition.timer().size() == 0, + DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, + () -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of()))); + // Acquisition lock timeout task has run already and next fetch offset is moved to 2. + assertEquals(2, sharePartition.nextFetchOffset()); + // Send the same batches again. + fetchAcquiredRecords(sharePartition, memoryRecords(5, 2), 5); + fetchAcquiredRecords(sharePartition, memoryRecords(5, 7), 5); + + future1 = new CompletableFuture<>(); + future2 = new CompletableFuture<>(); + Mockito.when(persister.writeState(Mockito.any())).thenReturn(future1).thenReturn(future2); + + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(3, 3, List.of(AcknowledgeType.ACCEPT.id)))); + sharePartition.acknowledge(MEMBER_ID, List.of(new ShareAcknowledgementBatch(7, 11, List.of(AcknowledgeType.ACCEPT.id)))); + + mockTimer.advanceClock(DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS); + // Verify the timer tasks have run and the state is archived for the offsets which are not acknowledged, + // but the acquisition lock timeout task should be just expired for acknowledged offsets, though + // the state should not be archived. + TestUtils.waitForCondition( + () -> sharePartition.cachedState().get(2L).offsetState().get(2L).state() == RecordState.ARCHIVED && + sharePartition.cachedState().get(2L).offsetState().get(3L).state() == RecordState.ACKNOWLEDGED && + sharePartition.cachedState().get(2L).offsetState().get(3L).acquisitionLockTimeoutTask().hasExpired() && + sharePartition.cachedState().get(7L).batchState() == RecordState.ACKNOWLEDGED && + sharePartition.cachedState().get(7L).batchAcquisitionLockTimeoutTask().hasExpired(), + DEFAULT_MAX_WAIT_ACQUISITION_LOCK_TIMEOUT_MS, + () -> assertionFailedMessage(sharePartition, Map.of(2L, List.of(3L), 7L, List.of()))); + + future1.complete(writeShareGroupStateResult); + // Now the state should be archived for the offsets despite the write state RPC failure, as the + // delivery count has reached the max delivery count and the acquisition lock timeout task + // has already expired for the offsets which were acknowledged. + assertEquals(12, sharePartition.nextFetchOffset()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(2L).offsetState().get(3L).state()); + assertEquals(2, sharePartition.cachedState().get(2L).offsetState().get(3L).deliveryCount()); + assertEquals(RecordState.ACKNOWLEDGED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(2, sharePartition.cachedState().get(7L).batchDeliveryCount()); + + future2.complete(writeShareGroupStateResult); + assertEquals(12L, sharePartition.nextFetchOffset()); + assertEquals(RecordState.ARCHIVED, sharePartition.cachedState().get(7L).batchState()); + assertEquals(2, sharePartition.cachedState().get(7L).batchDeliveryCount()); + } + /** * This function produces transactional data of a given no. of records followed by a transactional marker (COMMIT/ABORT). */ diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 43384a64789..d4bf2cacc8d 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -1489,6 +1489,18 @@ class KafkaConfigTest { assertEquals(expected, addresses) } + @Test + def testInvalidQuorumAutoJoinForKRaftBroker(): Unit = { + val props = TestUtils.createBrokerConfig(0) + props.setProperty(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, String.valueOf(true)) + assertEquals( + "requirement failed: controller.quorum.auto.join.enable is only " + + "supported when process.roles contains the 'controller' role.", + assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage + ) + + } + @Test def testAcceptsLargeId(): Unit = { val largeBrokerId = 2000 diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 35966996ce6..344df08f56b 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -385,6 +385,84 @@ Found problem: "Failed to find content in output: " + stream.toString()) } + @Test + def testFormatWithReleaseVersionAndFeatureOverride(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultStaticQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + assertEquals(0, runFormatCommand(stream, properties, Seq( + "--release-version", "3.7-IV0", + "--feature", "share.version=1"))) + + // Verify that the feature override is applied by checking the bootstrap metadata + val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString).read + + // Verify that the share.version feature is set to 1 as specified + assertEquals(1.toShort, bootstrapMetadata.featureLevel("share.version"), + "share.version should be set to 1") + + // Verify the command output contains the expected release version + assertTrue(stream.toString().contains("3.7-IV0"), + "Failed to find release version in output: " + stream.toString()) + + // Verify that the format command completed successfully with features + assertTrue(stream.toString().contains("Formatting metadata directory"), + "Failed to find formatting message in output: " + stream.toString()) + } + + @Test + def testFormatWithMultipleFeatures(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultStaticQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + assertEquals(0, runFormatCommand(stream, properties, Seq( + "--release-version", "3.8-IV0", + "--feature", "share.version=1", + "--feature", "transaction.version=2", + "--feature", "group.version=1"))) + + // Verify that all features are properly bootstrapped by checking the bootstrap metadata + val bootstrapMetadata = new BootstrapDirectory(availableDirs.head.toString).read + + // Verify that all specified features are set correctly + assertEquals(1.toShort, bootstrapMetadata.featureLevel("share.version"), + "share.version should be set to 1") + assertEquals(2.toShort, bootstrapMetadata.featureLevel("transaction.version"), + "transaction.version should be set to 2") + assertEquals(1.toShort, bootstrapMetadata.featureLevel("group.version"), + "group.version should be set to 1") + + // Verify the command output contains the expected release version + assertTrue(stream.toString().contains("3.8-IV0"), + "Failed to find release version in output: " + stream.toString()) + + // Verify that the format command completed successfully with multiple features + assertTrue(stream.toString().contains("Formatting metadata directory"), + "Failed to find formatting message in output: " + stream.toString()) + } + + @Test + def testFormatWithInvalidFeatureThrowsError(): Unit = { + val availableDirs = Seq(TestUtils.tempDir()) + val properties = new Properties() + properties.putAll(defaultStaticQuorumProperties) + properties.setProperty("log.dirs", availableDirs.mkString(",")) + val stream = new ByteArrayOutputStream() + + // Test with an invalid feature that doesn't exist + val exception = assertThrows(classOf[FormatterException], () => { + runFormatCommand(stream, properties, Seq( + "--release-version", "3.7-IV0", + "--feature", "stream.version=1")) + }) + + assertTrue(exception.getMessage.contains("Unsupported feature: stream.version.")) + } + @Test def testFormatWithStandaloneFlagOnBrokerFails(): Unit = { val availableDirs = Seq(TestUtils.tempDir()) diff --git a/docker/server.properties b/docker/server.properties index eb0b445c344..8ed486f3736 100644 --- a/docker/server.properties +++ b/docker/server.properties @@ -87,8 +87,8 @@ log.dirs=/tmp/kraft-combined-logs num.partitions=1 # The number of threads per data directory to be used for log recovery at startup and flushing at shutdown. -# This value is recommended to be increased for installations with data dirs located in RAID array. -num.recovery.threads.per.data.dir=1 +# This value is recommended to be increased based on the installation resources. +num.recovery.threads.per.data.dir=2 ############################# Internal Topic Settings ############################# # The replication factor for the group metadata internal topics "__consumer_offsets", "__share_group_state" and "__transaction_state" diff --git a/docs/configuration.html b/docs/configuration.html index 5eb79e46ecc..425d306a4dc 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -329,8 +329,23 @@

3.11 Tiered Storage Configs

Below is the Tiered Storage configuration. + +

3.11.1 RLMM Configs

+

Below is the configuration for TopicBasedRemoteLogMetadataManager, which is the default implementation of RemoteLogMetadataManager.

+

All configurations here should start with the prefix defined by remote.log.metadata.manager.impl.prefix, for example, rlmm.config.remote.log.metadata.consume.wait.ms.

+

The implementation of TopicBasedRemoteLogMetadataManager needs to create admin, producer, and consumer clients for the internal topic __remote_log_metadata.

+

Additional configurations can be provided for different types of clients using the following configuration properties:

+
# Configs for admin, producer, and consumer clients
+<rlmm.prefix>.remote.log.metadata.common.client.<kafka.property> = <value>
+
+# Configs only for producer client
+<rlmm.prefix>.remote.log.metadata.producer.<kafka.property> = <value>
+
+# Configs only for consumer client
+<rlmm.prefix>.remote.log.metadata.consumer.<kafka.property> = <value>
+

3.12 Configuration Providers diff --git a/docs/connect.html b/docs/connect.html index 457bc575eaa..aa3c2af3ea9 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -762,7 +762,7 @@ public List<SourceRecord> poll() throws InterruptedException {

The flush() method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The offsets parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the flush() operation atomically commits the data and offsets to a final location in HDFS.

-

Errant Record Reporter
+
Errant Record Reporter

When error reporting is enabled for a connector, the connector can use an ErrantRecordReporter to report problems with individual records sent to a sink connector. The following example shows how a connector's SinkTask subclass might obtain and use the ErrantRecordReporter, safely handling a null reporter when the DLQ is not enabled or when the connector is installed in an older Connect runtime that doesn't have this reporter feature:

diff --git a/docs/ops.html b/docs/ops.html index def641d82f1..61007a30b8f 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -2107,6 +2107,16 @@ The following set of metrics are available for monitoring the group coordinator: The max time in millis remote copies was throttled by a broker kafka.server:type=RemoteLogManager, name=remote-copy-throttle-time-max + + RemoteLogReader Fetch Rate And Time + The time to read data from remote storage by a broker + kafka.log.remote:type=RemoteLogManager,name=RemoteLogReaderFetchRateAndTimeMs + + + Delayed Remote List Offsets Expires Per Sec + The number of expired remote list offsets per second. Omitting 'topic=(...), partition=(...)' will yield the all-topic rate + kafka.server:type=DelayedRemoteListOffsetsMetrics,name=ExpiresPerSec,topic=([-.\w]+),partition=([0-9]+) + @@ -4501,9 +4511,16 @@ $ bin/kafka-topics.sh --create --topic tieredTopic --bootstrap-server localhost:

Downgrades are safe to perform by setting eligible.leader.replicas.version=0.

Tool

-

The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics. - Also note that, if min.insync.replicas is updated for a topic, the ELR field will be cleaned. If cluster default min ISR is updated, - all the ELR fields will be cleaned.

+

The ELR fields can be checked through the API DescribeTopicPartitions. The admin client can fetch the ELR info by describing the topics.

+

Note that when the ELR feature is enabled:

+
    +
  • The cluster-level min.insync.replicas config will be added if there is not any. The value is the same as the static config in the active controller.
  • +
  • The removal of min.insync.replicas config at the cluster-level is not allowed.
  • +
  • If the cluster-level min.insync.replicas is updated, even if the value is unchanged, all the ELR state will be cleaned.
  • +
  • The previously set min.insync.replicas value at the broker-level config will be removed. Please set at the cluster-level if necessary.
  • +
  • The alteration of min.insync.replicas config at the broker-level is not allowed.
  • +
  • If min.insync.replicas is updated for a topic, the ELR state will be cleaned.
  • +
diff --git a/docs/streams/core-concepts.html b/docs/streams/core-concepts.html index c400ca08453..a2d1b7209b5 100644 --- a/docs/streams/core-concepts.html +++ b/docs/streams/core-concepts.html @@ -279,7 +279,7 @@

In stream processing, one of the most frequently asked question is "does my stream processing system guarantee that each record is processed once and only once, even if some failures are encountered in the middle of processing?" Failing to guarantee exactly-once stream processing is a deal-breaker for many applications that cannot tolerate any data-loss or data duplicates, and in that case a batch-oriented framework is usually used in addition - to the stream processing pipeline, known as the Lambda Architecture. + to the stream processing pipeline, known as the Lambda Architecture. Prior to 0.11.0.0, Kafka only provides at-least-once delivery guarantees and hence any stream processing systems that leverage it as the backend storage could not guarantee end-to-end exactly-once semantics. In fact, even for those stream processing systems that claim to support exactly-once processing, as long as they are reading from / writing to Kafka as the source / sink, their applications cannot actually guarantee that no duplicates will be generated throughout the pipeline.
diff --git a/docs/toc.html b/docs/toc.html index 032228de3d2..304bd1c8a53 100644 --- a/docs/toc.html +++ b/docs/toc.html @@ -153,6 +153,7 @@

  • 6.7 Monitoring

    Upgrading to 4.1.0

    @@ -57,6 +60,9 @@ The logger class name for LogCleaner has been updated from kafka.log.LogCleaner to org.apache.kafka.storage.internals.log.LogCleaner in the log4j2.yaml configuration file. Added loggers for org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread and org.apache.kafka.storage.internals.log.Cleaner classes to CleanerAppender.
  • +
  • + The filename for rotated state-change.log files has been updated from stage-change.log.[date] to state-change.log.[date] in the log4j2.yaml configuration file. +
  • Broker @@ -68,6 +74,8 @@
  • The KIP-966 part 1: Eligible Leader Replicas(ELR) will be enabled by default on the new clusters. + After the ELR feature enabled, the previously set min.insync.replicas value at the broker-level config will be removed. + Please set at the cluster-level if necessary. For further details, please refer to here.
  • @@ -102,9 +110,9 @@ -

    Upgrading to 4.0.0

    +

    Upgrading to 4.0.1

    -
    Upgrading Clients to 4.0.0
    +
    Upgrading Clients to 4.0.1

    For a rolling upgrade:

    @@ -115,7 +123,7 @@ or KIP-1124. -
    Upgrading Servers to 4.0.0 from any version 3.3.x through 3.9.x
    +
    Upgrading Servers to 4.0.1 from any version 3.3.x through 3.9.x

    Note: Apache Kafka 4.0 only supports KRaft mode - ZooKeeper mode has been removed. As such, broker upgrades to 4.0.0 (and higher) require KRaft mode and the software and metadata versions must be at least 3.3.x (the first version when KRaft mode was deemed production ready). For clusters in KRaft mode @@ -139,7 +147,13 @@ has a boolean parameter that indicates if there are metadata changes (i.e. IBP_4_0_IV1(23, "4.0", "IV1", true) means this version has metadata changes). Given your current and target versions, a downgrade is only possible if there are no metadata changes in the versions between. - +

    Notable changes in 4.0.1
    +
      +
    • + The filename for rotated state-change.log files has been updated from stage-change.log.[date] to state-change.log.[date] in the log4j2.yaml configuration file. + See KAFKA-19576 for details. +
    • +
    Notable changes in 4.0.0
    +
  • + The filename for rotated state-change.log files incorrectly rotates to stage-change.log.[date] (changing state to stage). This issue is corrected in 4.0.1. + See KAFKA-19576 for details. +
  • diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 04119d57704..bd957d7f8a7 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -123,7 +123,7 @@ versions += [ scoverage: "2.0.11", slf4j: "1.7.36", snappy: "1.1.10.7", - spotbugs: "4.8.6", + spotbugs: "4.9.4", mockOAuth2Server: "2.2.1", zinc: "1.9.2", // When updating the zstd version, please do as well in docker/native/native-image-configs/resource-config.json diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml index 080681e8db3..22a08ebd051 100644 --- a/gradle/spotbugs-exclude.xml +++ b/gradle/spotbugs-exclude.xml @@ -550,4 +550,189 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java index d0e48f18142..4a0219d1759 100644 --- a/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java +++ b/metadata/src/main/java/org/apache/kafka/controller/ConfigurationControlManager.java @@ -611,7 +611,7 @@ public class ConfigurationControlManager { /** * Generate any configuration records that are needed to make it safe to enable ELR. - * Specifically, we need to remove all cluster-level configurations for min.insync.replicas, + * Specifically, we need to remove all broker-level configurations for min.insync.replicas, * and create a cluster-level configuration for min.insync.replicas. It is always safe to call * this function if ELR is already enabled; it will simply do nothing if the necessary * configurations already exist. diff --git a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java index b5ae7068988..553fc54c07b 100644 --- a/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java +++ b/metadata/src/main/java/org/apache/kafka/image/TopicDelta.java @@ -211,11 +211,7 @@ public final class TopicDelta { try { PartitionRegistration prevPartition = image.partitions().get(entry.getKey()); - if ( - prevPartition == null || - prevPartition.directories == null || - prevPartition.directory(brokerId) != entry.getValue().directory(brokerId) - ) { + if (prevPartition == null || prevPartition.directory(brokerId) != entry.getValue().directory(brokerId)) { directoryIds.put( new TopicIdPartition(id(), new TopicPartition(name(), entry.getKey())), entry.getValue().directory(brokerId) diff --git a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java index 38219c2069a..3f7cba37724 100644 --- a/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java +++ b/metadata/src/main/java/org/apache/kafka/metadata/storage/Formatter.java @@ -415,6 +415,7 @@ public class Formatter { if (ensemble.emptyLogDirs().isEmpty()) { printStream.println("All of the log directories are already formatted."); } else { + printStream.println("Bootstrap metadata: " + bootstrapMetadata); Map directoryTypes = new HashMap<>(); for (String emptyLogDir : ensemble.emptyLogDirs()) { DirectoryType directoryType = DirectoryType.calculate(emptyLogDir, diff --git a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java index 5ddcd2d8889..d4e13b4ccab 100644 --- a/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java +++ b/metadata/src/test/java/org/apache/kafka/metadata/storage/FormatterTest.java @@ -89,7 +89,7 @@ public class FormatterTest { Formatter formatter = new Formatter(). setNodeId(DEFAULT_NODE_ID). setClusterId(DEFAULT_CLUSTER_ID.toString()); - directories.forEach(d -> formatter.addDirectory(d)); + directories.forEach(formatter::addDirectory); formatter.setMetadataLogDirectory(directories.get(0)); return new FormatterContext(formatter); } @@ -170,7 +170,7 @@ public class FormatterTest { String expectedPrefix = "Error while writing meta.properties file"; assertEquals(expectedPrefix, assertThrows(FormatterException.class, - () -> formatter1.formatter.run()). + formatter1.formatter::run). getMessage().substring(0, expectedPrefix.length())); } } @@ -180,15 +180,15 @@ public class FormatterTest { try (TestEnv testEnv = new TestEnv(1)) { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.run(); - assertEquals("Formatting metadata directory " + testEnv.directory(0) + - " with metadata.version " + MetadataVersion.latestProduction() + ".", - formatter1.output().trim()); + assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + + "\nFormatting metadata directory " + testEnv.directory(0) + + " with metadata.version " + MetadataVersion.latestProduction() + ".", + formatter1.output().trim()); FormatterContext formatter2 = testEnv.newFormatter(); formatter2.formatter.setIgnoreFormatted(true); formatter2.formatter.run(); - assertEquals("All of the log directories are already formatted.", - formatter2.output().trim()); + assertTrue(formatter2.output().trim().contains("All of the log directories are already formatted.")); } } @@ -201,7 +201,8 @@ public class FormatterTest { formatter1.formatter .setInitialControllers(DynamicVoters.parse("1@localhost:8020:" + originalDirectoryId)) .run(); - assertEquals("Formatting dynamic metadata voter directory " + testEnv.directory(0) + + assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + + "\nFormatting dynamic metadata voter directory " + testEnv.directory(0) + " with metadata.version " + MetadataVersion.latestProduction() + ".", formatter1.output().trim()); assertMetadataDirectoryId(testEnv, Uuid.fromString(originalDirectoryId)); @@ -244,9 +245,10 @@ public class FormatterTest { FormatterContext formatter2 = testEnv.newFormatter(); formatter2.formatter.setIgnoreFormatted(true); formatter2.formatter.run(); - assertEquals("Formatting data directory " + testEnv.directory(1) + " with metadata.version " + - MetadataVersion.latestProduction() + ".", - formatter2.output().trim()); + assertEquals("Bootstrap metadata: " + formatter2.formatter.bootstrapMetadata() + + "\nFormatting data directory " + testEnv.directory(1) + " with metadata.version " + + MetadataVersion.latestProduction() + ".", + formatter2.output().trim()); } } @@ -256,9 +258,10 @@ public class FormatterTest { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.IBP_3_5_IV0); formatter1.formatter.run(); - assertEquals("Formatting metadata directory " + testEnv.directory(0) + - " with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".", - formatter1.output().trim()); + assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + + "\nFormatting metadata directory " + testEnv.directory(0) + + " with metadata.version " + MetadataVersion.IBP_3_5_IV0 + ".", + formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.IBP_3_5_IV0, bootstrapMetadata.metadataVersion()); @@ -272,7 +275,7 @@ public class FormatterTest { FormatterContext formatter1 = testEnv.newFormatter(); formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting()); assertEquals("metadata.version " + MetadataVersion.latestTesting() + " is not yet stable.", - assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage()); + assertThrows(FormatterException.class, formatter1.formatter::run).getMessage()); } } @@ -283,9 +286,10 @@ public class FormatterTest { formatter1.formatter.setReleaseVersion(MetadataVersion.latestTesting()); formatter1.formatter.setUnstableFeatureVersionsEnabled(true); formatter1.formatter.run(); - assertEquals("Formatting metadata directory " + testEnv.directory(0) + - " with metadata.version " + MetadataVersion.latestTesting() + ".", - formatter1.output().trim()); + assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + + "\nFormatting metadata directory " + testEnv.directory(0) + + " with metadata.version " + MetadataVersion.latestTesting() + ".", + formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.latestTesting(), bootstrapMetadata.metadataVersion()); @@ -318,7 +322,7 @@ public class FormatterTest { "saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]")); assertEquals("SCRAM is only supported in metadata.version 3.5-IV2 or later.", assertThrows(FormatterException.class, - () -> formatter1.formatter.run()).getMessage()); + formatter1.formatter::run).getMessage()); } } @@ -333,9 +337,10 @@ public class FormatterTest { "SCRAM-SHA-512=[name=alice,salt=\"MWx2NHBkbnc0ZndxN25vdGN4bTB5eTFrN3E=\"," + "saltedpassword=\"mT0yyUUxnlJaC99HXgRTSYlbuqa4FSGtJCJfTMvjYCE=\"]")); formatter1.formatter.run(); - assertEquals("Formatting metadata directory " + testEnv.directory(0) + - " with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".", - formatter1.output().trim()); + assertEquals("Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata() + + "\nFormatting metadata directory " + testEnv.directory(0) + + " with metadata.version " + MetadataVersion.IBP_3_8_IV0 + ".", + formatter1.output().trim()); BootstrapMetadata bootstrapMetadata = new BootstrapDirectory(testEnv.directory(0)).read(); assertEquals(MetadataVersion.IBP_3_8_IV0, bootstrapMetadata.metadataVersion()); @@ -406,7 +411,7 @@ public class FormatterTest { "are: eligible.leader.replicas.version, group.version, kraft.version, " + "share.version, streams.version, test.feature.version, transaction.version", assertThrows(FormatterException.class, - () -> formatter1.formatter.run()). + formatter1.formatter::run). getMessage()); } } @@ -425,6 +430,7 @@ public class FormatterTest { formatter1.formatter.run(); assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); assertEquals(List.of( + "Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(), String.format("Formatting data directory %s with %s %s.", testEnv.directory(1), MetadataVersion.FEATURE_NAME, @@ -459,7 +465,7 @@ public class FormatterTest { "Cannot set kraft.version to 0 if one of the flags --standalone, --initial-controllers, or " + "--no-initial-controllers is used. For dynamic controllers support, try removing the " + "--feature flag for kraft.version.", - assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage() + assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() ); } } @@ -475,7 +481,7 @@ public class FormatterTest { "Cannot set kraft.version to 1 unless one of the flags --standalone, --initial-controllers, or " + "--no-initial-controllers is used. For dynamic controllers support, try using one of " + "--standalone, --initial-controllers, or --no-initial-controllers.", - assertThrows(FormatterException.class, () -> formatter1.formatter.run()).getMessage() + assertThrows(FormatterException.class, formatter1.formatter::run).getMessage() ); } } @@ -492,7 +498,7 @@ public class FormatterTest { assertEquals("kraft.version could not be set to 1 because it depends on " + "metadata.version level 21", assertThrows(IllegalArgumentException.class, - () -> formatter1.formatter.run()).getMessage()); + formatter1.formatter::run).getMessage()); } } @@ -514,12 +520,12 @@ public class FormatterTest { formatter1.formatter.setInitialControllers(DynamicVoters. parse("1@localhost:8020:4znU-ou9Taa06bmEJxsjnw")); if (metadataVersion.isAtLeast(MetadataVersion.IBP_4_0_IV1)) { - assertDoesNotThrow(() -> formatter1.formatter.run()); + assertDoesNotThrow(formatter1.formatter::run); } else { assertEquals("eligible.leader.replicas.version could not be set to 1 because it depends on " + "metadata.version level 23", assertThrows(IllegalArgumentException.class, - () -> formatter1.formatter.run()).getMessage()); + formatter1.formatter::run).getMessage()); } } } @@ -539,6 +545,7 @@ public class FormatterTest { formatter1.formatter.run(); assertEquals((short) 1, formatter1.formatter.featureLevels.getOrDefault("kraft.version", (short) 0)); assertEquals(List.of( + "Bootstrap metadata: " + formatter1.formatter.bootstrapMetadata(), String.format("Formatting data directory %s with %s %s.", testEnv.directory(1), MetadataVersion.FEATURE_NAME, diff --git a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java index 09675f38082..4cbc8778149 100644 --- a/raft/src/main/java/org/apache/kafka/raft/FollowerState.java +++ b/raft/src/main/java/org/apache/kafka/raft/FollowerState.java @@ -40,8 +40,8 @@ public class FollowerState implements EpochState { private final Set voters; // Used for tracking the expiration of both the Fetch and FetchSnapshot requests private final Timer fetchTimer; - // Used to track when to send another update voter request - private final Timer updateVoterPeriodTimer; + // Used to track when to send another add, remove, or update voter request + private final Timer updateVoterSetPeriodTimer; /* Used to track if the replica has fetched successfully from the leader at least once since * the transition to follower in this epoch. If the replica has not yet fetched successfully, @@ -76,7 +76,7 @@ public class FollowerState implements EpochState { this.votedKey = votedKey; this.voters = voters; this.fetchTimer = time.timer(fetchTimeoutMs); - this.updateVoterPeriodTimer = time.timer(updateVoterPeriodMs()); + this.updateVoterSetPeriodTimer = time.timer(updateVoterPeriodMs()); this.highWatermark = highWatermark; this.log = logContext.logger(FollowerState.class); } @@ -154,19 +154,19 @@ public class FollowerState implements EpochState { return fetchTimeoutMs; } - public boolean hasUpdateVoterPeriodExpired(long currentTimeMs) { - updateVoterPeriodTimer.update(currentTimeMs); - return updateVoterPeriodTimer.isExpired(); + public boolean hasUpdateVoterSetPeriodExpired(long currentTimeMs) { + updateVoterSetPeriodTimer.update(currentTimeMs); + return updateVoterSetPeriodTimer.isExpired(); } - public long remainingUpdateVoterPeriodMs(long currentTimeMs) { - updateVoterPeriodTimer.update(currentTimeMs); - return updateVoterPeriodTimer.remainingMs(); + public long remainingUpdateVoterSetPeriodMs(long currentTimeMs) { + updateVoterSetPeriodTimer.update(currentTimeMs); + return updateVoterSetPeriodTimer.remainingMs(); } - public void resetUpdateVoterPeriod(long currentTimeMs) { - updateVoterPeriodTimer.update(currentTimeMs); - updateVoterPeriodTimer.reset(updateVoterPeriodMs()); + public void resetUpdateVoterSetPeriod(long currentTimeMs) { + updateVoterSetPeriodTimer.update(currentTimeMs); + updateVoterSetPeriodTimer.reset(updateVoterPeriodMs()); } public boolean hasUpdatedLeader() { diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java index 2a88c2a7830..898a82ef3fd 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaNetworkChannel.java @@ -19,11 +19,13 @@ package org.apache.kafka.raft; import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.KafkaClient; import org.apache.kafka.common.Node; +import org.apache.kafka.common.message.AddRaftVoterRequestData; import org.apache.kafka.common.message.ApiVersionsRequestData; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.EndQuorumEpochRequestData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchSnapshotRequestData; +import org.apache.kafka.common.message.RemoveRaftVoterRequestData; import org.apache.kafka.common.message.UpdateRaftVoterRequestData; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.network.ListenerName; @@ -31,11 +33,13 @@ import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AddRaftVoterRequest; import org.apache.kafka.common.requests.ApiVersionsRequest; import org.apache.kafka.common.requests.BeginQuorumEpochRequest; import org.apache.kafka.common.requests.EndQuorumEpochRequest; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchSnapshotRequest; +import org.apache.kafka.common.requests.RemoveRaftVoterRequest; import org.apache.kafka.common.requests.UpdateRaftVoterRequest; import org.apache.kafka.common.requests.VoteRequest; import org.apache.kafka.common.utils.Time; @@ -181,20 +185,25 @@ public class KafkaNetworkChannel implements NetworkChannel { static AbstractRequest.Builder buildRequest(ApiMessage requestData) { if (requestData instanceof VoteRequestData) return new VoteRequest.Builder((VoteRequestData) requestData); - if (requestData instanceof BeginQuorumEpochRequestData) + else if (requestData instanceof BeginQuorumEpochRequestData) return new BeginQuorumEpochRequest.Builder((BeginQuorumEpochRequestData) requestData); - if (requestData instanceof EndQuorumEpochRequestData) + else if (requestData instanceof EndQuorumEpochRequestData) return new EndQuorumEpochRequest.Builder((EndQuorumEpochRequestData) requestData); - if (requestData instanceof FetchRequestData) + else if (requestData instanceof FetchRequestData) return new FetchRequest.SimpleBuilder((FetchRequestData) requestData); - if (requestData instanceof FetchSnapshotRequestData) + else if (requestData instanceof FetchSnapshotRequestData) return new FetchSnapshotRequest.Builder((FetchSnapshotRequestData) requestData); - if (requestData instanceof UpdateRaftVoterRequestData) + else if (requestData instanceof UpdateRaftVoterRequestData) return new UpdateRaftVoterRequest.Builder((UpdateRaftVoterRequestData) requestData); - if (requestData instanceof ApiVersionsRequestData) + else if (requestData instanceof AddRaftVoterRequestData) + return new AddRaftVoterRequest.Builder((AddRaftVoterRequestData) requestData); + else if (requestData instanceof RemoveRaftVoterRequestData) + return new RemoveRaftVoterRequest.Builder((RemoveRaftVoterRequestData) requestData); + else if (requestData instanceof ApiVersionsRequestData) return new ApiVersionsRequest.Builder((ApiVersionsRequestData) requestData, ApiKeys.API_VERSIONS.oldestVersion(), ApiKeys.API_VERSIONS.latestVersion()); - throw new IllegalArgumentException("Unexpected type for requestData: " + requestData); + else + throw new IllegalArgumentException("Unexpected type for requestData: " + requestData); } } diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java index 921bd72ecf2..bb70c5a7df4 100644 --- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java @@ -180,7 +180,7 @@ public final class KafkaRaftClient implements RaftClient { private final Logger logger; private final Time time; private final int fetchMaxWaitMs; - private final boolean followersAlwaysFlush; + private final boolean canBecomeVoter; private final String clusterId; private final Endpoints localListeners; private final SupportedVersionRange localSupportedKRaftVersion; @@ -229,7 +229,7 @@ public final class KafkaRaftClient implements RaftClient { * non-participating observer. * * @param nodeDirectoryId the node directory id, cannot be the zero uuid - * @param followersAlwaysFlush instruct followers to always fsync when appending to the log + * @param canBecomeVoter instruct followers to always fsync when appending to the log */ public KafkaRaftClient( OptionalInt nodeId, @@ -240,7 +240,7 @@ public final class KafkaRaftClient implements RaftClient { Time time, ExpirationService expirationService, LogContext logContext, - boolean followersAlwaysFlush, + boolean canBecomeVoter, String clusterId, Collection bootstrapServers, Endpoints localListeners, @@ -258,7 +258,7 @@ public final class KafkaRaftClient implements RaftClient { time, expirationService, MAX_FETCH_WAIT_MS, - followersAlwaysFlush, + canBecomeVoter, clusterId, bootstrapServers, localListeners, @@ -280,7 +280,7 @@ public final class KafkaRaftClient implements RaftClient { Time time, ExpirationService expirationService, int fetchMaxWaitMs, - boolean followersAlwaysFlush, + boolean canBecomeVoter, String clusterId, Collection bootstrapServers, Endpoints localListeners, @@ -308,7 +308,7 @@ public final class KafkaRaftClient implements RaftClient { this.localListeners = localListeners; this.localSupportedKRaftVersion = localSupportedKRaftVersion; this.fetchMaxWaitMs = fetchMaxWaitMs; - this.followersAlwaysFlush = followersAlwaysFlush; + this.canBecomeVoter = canBecomeVoter; this.logger = logContext.logger(KafkaRaftClient.class); this.random = random; this.quorumConfig = quorumConfig; @@ -1839,7 +1839,7 @@ public final class KafkaRaftClient implements RaftClient { ); } - if (quorum.isVoter() || followersAlwaysFlush) { + if (quorum.isVoter() || canBecomeVoter) { // the leader only requires that voters have flushed their log before sending a Fetch // request. Because of reconfiguration some observers (that are getting added to the // voter set) need to flush the disk because the leader may assume that they are in the @@ -2291,6 +2291,25 @@ public final class KafkaRaftClient implements RaftClient { ); } + private boolean handleAddVoterResponse( + RaftResponse.Inbound responseMetadata, + long currentTimeMs + ) { + final AddRaftVoterResponseData data = (AddRaftVoterResponseData) responseMetadata.data(); + final Errors error = Errors.forCode(data.errorCode()); + + /* These error codes indicate the replica was successfully added or the leader is unable to + * process the request. In either case, reset the update voter set timer to back off. + */ + if (error == Errors.NONE || error == Errors.REQUEST_TIMED_OUT || + error == Errors.DUPLICATE_VOTER) { + quorum.followerStateOrThrow().resetUpdateVoterSetPeriod(currentTimeMs); + return true; + } else { + return handleUnexpectedError(error, responseMetadata); + } + } + private CompletableFuture handleRemoveVoterRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs @@ -2334,6 +2353,25 @@ public final class KafkaRaftClient implements RaftClient { ); } + private boolean handleRemoveVoterResponse( + RaftResponse.Inbound responseMetadata, + long currentTimeMs + ) { + final RemoveRaftVoterResponseData data = (RemoveRaftVoterResponseData) responseMetadata.data(); + final Errors error = Errors.forCode(data.errorCode()); + + /* These error codes indicate the replica was successfully removed or the leader is unable to + * process the request. In either case, reset the update voter set timer to back off. + */ + if (error == Errors.NONE || error == Errors.REQUEST_TIMED_OUT || + error == Errors.VOTER_NOT_FOUND) { + quorum.followerStateOrThrow().resetUpdateVoterSetPeriod(currentTimeMs); + return true; + } else { + return handleUnexpectedError(error, responseMetadata); + } + } + private CompletableFuture handleUpdateVoterRequest( RaftRequest.Inbound requestMetadata, long currentTimeMs @@ -2629,6 +2667,14 @@ public final class KafkaRaftClient implements RaftClient { handledSuccessfully = handleUpdateVoterResponse(response, currentTimeMs); break; + case ADD_RAFT_VOTER: + handledSuccessfully = handleAddVoterResponse(response, currentTimeMs); + break; + + case REMOVE_RAFT_VOTER: + handledSuccessfully = handleRemoveVoterResponse(response, currentTimeMs); + break; + default: throw new IllegalArgumentException("Received unexpected response type: " + apiKey); } @@ -3247,7 +3293,7 @@ public final class KafkaRaftClient implements RaftClient { logger.info("Transitioning to Prospective state due to fetch timeout"); transitionToProspective(currentTimeMs); backoffMs = 0; - } else if (state.hasUpdateVoterPeriodExpired(currentTimeMs)) { + } else if (state.hasUpdateVoterSetPeriodExpired(currentTimeMs)) { final boolean resetUpdateVoterTimer; if (shouldSendUpdateVoteRequest(state)) { var sendResult = maybeSendUpdateVoterRequest(state, currentTimeMs); @@ -3261,7 +3307,7 @@ public final class KafkaRaftClient implements RaftClient { } if (resetUpdateVoterTimer) { - state.resetUpdateVoterPeriod(currentTimeMs); + state.resetUpdateVoterSetPeriod(currentTimeMs); } } else { backoffMs = maybeSendFetchToBestNode(state, currentTimeMs); @@ -3271,13 +3317,56 @@ public final class KafkaRaftClient implements RaftClient { backoffMs, Math.min( state.remainingFetchTimeMs(currentTimeMs), - state.remainingUpdateVoterPeriodMs(currentTimeMs) + state.remainingUpdateVoterSetPeriodMs(currentTimeMs) ) ); } + private boolean shouldSendAddOrRemoveVoterRequest(FollowerState state, long currentTimeMs) { + /* When the cluster supports reconfiguration, only replicas that can become a voter + * and are configured to auto join should attempt to automatically join the voter + * set for the configured topic partition. + */ + return partitionState.lastKraftVersion().isReconfigSupported() && canBecomeVoter && + quorumConfig.autoJoin() && state.hasUpdateVoterSetPeriodExpired(currentTimeMs); + } + private long pollFollowerAsObserver(FollowerState state, long currentTimeMs) { - return maybeSendFetchToBestNode(state, currentTimeMs); + GracefulShutdown shutdown = this.shutdown.get(); + final long backoffMs; + if (shutdown != null) { + // If we are an observer, then we can shutdown immediately. We want to + // skip potentially sending any add or remove voter RPCs. + backoffMs = 0; + } else if (shouldSendAddOrRemoveVoterRequest(state, currentTimeMs)) { + final var localReplicaKey = quorum.localReplicaKeyOrThrow(); + final var voters = partitionState.lastVoterSet(); + final RequestSendResult sendResult; + if (voters.voterIds().contains(localReplicaKey.id())) { + /* The replica's id is in the voter set but the replica is not a voter because + * the directory id of the voter set entry is different. Remove the old voter. + * Local replica is not in the voter set because the replica is an observer. + */ + final var oldVoter = voters.voterKeys() + .stream() + .filter(replicaKey -> replicaKey.id() == localReplicaKey.id()) + .findFirst() + .get(); + sendResult = maybeSendRemoveVoterRequest(state, oldVoter, currentTimeMs); + } else { + sendResult = maybeSendAddVoterRequest(state, currentTimeMs); + } + backoffMs = sendResult.timeToWaitMs(); + if (sendResult.requestSent()) { + state.resetUpdateVoterSetPeriod(currentTimeMs); + } + } else { + backoffMs = maybeSendFetchToBestNode(state, currentTimeMs); + } + return Math.min( + backoffMs, + state.remainingUpdateVoterSetPeriodMs(currentTimeMs) + ); } private long maybeSendFetchToBestNode(FollowerState state, long currentTimeMs) { @@ -3329,6 +3418,23 @@ public final class KafkaRaftClient implements RaftClient { ); } + private AddRaftVoterRequestData buildAddVoterRequest() { + return RaftUtil.addVoterRequest( + clusterId, + quorumConfig.requestTimeoutMs(), + quorum.localReplicaKeyOrThrow(), + localListeners, + false + ); + } + + private RemoveRaftVoterRequestData buildRemoveVoterRequest(ReplicaKey replicaKey) { + return RaftUtil.removeVoterRequest( + clusterId, + replicaKey + ); + } + private RequestSendResult maybeSendUpdateVoterRequest(FollowerState state, long currentTimeMs) { return maybeSendRequest( currentTimeMs, @@ -3337,6 +3443,29 @@ public final class KafkaRaftClient implements RaftClient { ); } + private RequestSendResult maybeSendAddVoterRequest( + FollowerState state, + long currentTimeMs + ) { + return maybeSendRequest( + currentTimeMs, + state.leaderNode(channel.listenerName()), + this::buildAddVoterRequest + ); + } + + private RequestSendResult maybeSendRemoveVoterRequest( + FollowerState state, + ReplicaKey replicaKey, + long currentTimeMs + ) { + return maybeSendRequest( + currentTimeMs, + state.leaderNode(channel.listenerName()), + () -> buildRemoveVoterRequest(replicaKey) + ); + } + private long pollUnattached(long currentTimeMs) { UnattachedState state = quorum.unattachedStateOrThrow(); if (quorum.isVoter()) { diff --git a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java index 3ff2f7c86de..3712c1cc92d 100644 --- a/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java +++ b/raft/src/main/java/org/apache/kafka/raft/QuorumConfig.java @@ -35,6 +35,7 @@ import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH; import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; +import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.INT; import static org.apache.kafka.common.config.ConfigDef.Type.LIST; @@ -102,6 +103,11 @@ public class QuorumConfig { public static final String QUORUM_RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; public static final int DEFAULT_QUORUM_RETRY_BACKOFF_MS = 20; + public static final String QUORUM_AUTO_JOIN_ENABLE_CONFIG = QUORUM_PREFIX + "auto.join.enable"; + public static final String QUORUM_AUTO_JOIN_ENABLE_DOC = "Controls whether a KRaft controller should automatically " + + "join the cluster metadata partition for its cluster id."; + public static final boolean DEFAULT_QUORUM_AUTO_JOIN_ENABLE = false; + public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(QUORUM_VOTERS_CONFIG, LIST, DEFAULT_QUORUM_VOTERS, new ControllerQuorumVotersValidator(), HIGH, QUORUM_VOTERS_DOC) .define(QUORUM_BOOTSTRAP_SERVERS_CONFIG, LIST, DEFAULT_QUORUM_BOOTSTRAP_SERVERS, new ControllerQuorumBootstrapServersValidator(), HIGH, QUORUM_BOOTSTRAP_SERVERS_DOC) @@ -110,7 +116,8 @@ public class QuorumConfig { .define(QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS, atLeast(0), HIGH, QUORUM_ELECTION_BACKOFF_MAX_MS_DOC) .define(QUORUM_LINGER_MS_CONFIG, INT, DEFAULT_QUORUM_LINGER_MS, atLeast(0), MEDIUM, QUORUM_LINGER_MS_DOC) .define(QUORUM_REQUEST_TIMEOUT_MS_CONFIG, INT, DEFAULT_QUORUM_REQUEST_TIMEOUT_MS, atLeast(0), MEDIUM, QUORUM_REQUEST_TIMEOUT_MS_DOC) - .define(QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, DEFAULT_QUORUM_RETRY_BACKOFF_MS, atLeast(0), LOW, QUORUM_RETRY_BACKOFF_MS_DOC); + .define(QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, DEFAULT_QUORUM_RETRY_BACKOFF_MS, atLeast(0), LOW, QUORUM_RETRY_BACKOFF_MS_DOC) + .define(QUORUM_AUTO_JOIN_ENABLE_CONFIG, BOOLEAN, DEFAULT_QUORUM_AUTO_JOIN_ENABLE, LOW, QUORUM_AUTO_JOIN_ENABLE_DOC); private final List voters; private final List bootstrapServers; @@ -120,6 +127,7 @@ public class QuorumConfig { private final int electionBackoffMaxMs; private final int fetchTimeoutMs; private final int appendLingerMs; + private final boolean autoJoin; public QuorumConfig(AbstractConfig abstractConfig) { this.voters = abstractConfig.getList(QUORUM_VOTERS_CONFIG); @@ -130,6 +138,7 @@ public class QuorumConfig { this.electionBackoffMaxMs = abstractConfig.getInt(QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG); this.fetchTimeoutMs = abstractConfig.getInt(QUORUM_FETCH_TIMEOUT_MS_CONFIG); this.appendLingerMs = abstractConfig.getInt(QUORUM_LINGER_MS_CONFIG); + this.autoJoin = abstractConfig.getBoolean(QUORUM_AUTO_JOIN_ENABLE_CONFIG); } public List voters() { @@ -164,6 +173,10 @@ public class QuorumConfig { return appendLingerMs; } + public boolean autoJoin() { + return autoJoin; + } + private static Integer parseVoterId(String idString) { try { return Integer.parseInt(idString); diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java index 273a2ec56c5..d2d59888178 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftClient.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftClient.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.raft; +import org.apache.kafka.common.errors.ApiException; import org.apache.kafka.raft.errors.BufferAllocationException; import org.apache.kafka.raft.errors.NotLeaderException; import org.apache.kafka.server.common.KRaftVersion; diff --git a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java index caa087378c5..f3f411885a7 100644 --- a/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java +++ b/raft/src/main/java/org/apache/kafka/raft/RaftUtil.java @@ -64,6 +64,8 @@ public class RaftUtil { case FETCH_SNAPSHOT -> new FetchSnapshotResponseData().setErrorCode(error.code()); case API_VERSIONS -> new ApiVersionsResponseData().setErrorCode(error.code()); case UPDATE_RAFT_VOTER -> new UpdateRaftVoterResponseData().setErrorCode(error.code()); + case ADD_RAFT_VOTER -> new AddRaftVoterResponseData().setErrorCode(error.code()); + case REMOVE_RAFT_VOTER -> new RemoveRaftVoterResponseData().setErrorCode(error.code()); default -> throw new IllegalArgumentException("Received response for unexpected request type: " + apiKey); }; } @@ -524,14 +526,16 @@ public class RaftUtil { String clusterId, int timeoutMs, ReplicaKey voter, - Endpoints listeners + Endpoints listeners, + boolean ackWhenCommitted ) { return new AddRaftVoterRequestData() .setClusterId(clusterId) .setTimeoutMs(timeoutMs) .setVoterId(voter.id()) .setVoterDirectoryId(voter.directoryId().orElse(ReplicaKey.NO_DIRECTORY_ID)) - .setListeners(listeners.toAddVoterRequest()); + .setListeners(listeners.toAddVoterRequest()) + .setAckWhenCommitted(ackWhenCommitted); } public static AddRaftVoterResponseData addVoterResponse( diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java index e56b9c94b49..8b2e70d8a2a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaNetworkChannelTest.java @@ -22,12 +22,14 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; import org.apache.kafka.common.feature.SupportedVersionRange; +import org.apache.kafka.common.message.AddRaftVoterResponseData; import org.apache.kafka.common.message.ApiVersionsResponseData; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; import org.apache.kafka.common.message.EndQuorumEpochResponseData; import org.apache.kafka.common.message.FetchRequestData; import org.apache.kafka.common.message.FetchResponseData; import org.apache.kafka.common.message.FetchSnapshotResponseData; +import org.apache.kafka.common.message.RemoveRaftVoterResponseData; import org.apache.kafka.common.message.UpdateRaftVoterResponseData; import org.apache.kafka.common.message.VoteResponseData; import org.apache.kafka.common.network.ListenerName; @@ -36,6 +38,7 @@ import org.apache.kafka.common.protocol.ApiMessage; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractResponse; +import org.apache.kafka.common.requests.AddRaftVoterResponse; import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.BeginQuorumEpochRequest; import org.apache.kafka.common.requests.BeginQuorumEpochResponse; @@ -44,6 +47,7 @@ import org.apache.kafka.common.requests.EndQuorumEpochResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.FetchSnapshotResponse; +import org.apache.kafka.common.requests.RemoveRaftVoterResponse; import org.apache.kafka.common.requests.UpdateRaftVoterResponse; import org.apache.kafka.common.requests.VoteRequest; import org.apache.kafka.common.requests.VoteResponse; @@ -88,7 +92,9 @@ public class KafkaNetworkChannelTest { ApiKeys.END_QUORUM_EPOCH, ApiKeys.FETCH, ApiKeys.FETCH_SNAPSHOT, - ApiKeys.UPDATE_RAFT_VOTER + ApiKeys.UPDATE_RAFT_VOTER, + ApiKeys.ADD_RAFT_VOTER, + ApiKeys.REMOVE_RAFT_VOTER ); private final int requestTimeoutMs = 30000; @@ -316,6 +322,21 @@ public class KafkaNetworkChannelTest { Endpoints.empty() ); + case ADD_RAFT_VOTER: + return RaftUtil.addVoterRequest( + clusterId, + requestTimeoutMs, + ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID), + Endpoints.empty(), + true + ); + + case REMOVE_RAFT_VOTER: + return RaftUtil.removeVoterRequest( + clusterId, + ReplicaKey.of(1, ReplicaKey.NO_DIRECTORY_ID) + ); + default: throw new AssertionError("Unexpected api " + key); } @@ -345,6 +366,8 @@ public class KafkaNetworkChannelTest { case FETCH -> new FetchResponseData().setErrorCode(error.code()); case FETCH_SNAPSHOT -> new FetchSnapshotResponseData().setErrorCode(error.code()); case UPDATE_RAFT_VOTER -> new UpdateRaftVoterResponseData().setErrorCode(error.code()); + case ADD_RAFT_VOTER -> new AddRaftVoterResponseData().setErrorCode(error.code()); + case REMOVE_RAFT_VOTER -> new RemoveRaftVoterResponseData().setErrorCode(error.code()); default -> throw new AssertionError("Unexpected api " + key); }; } @@ -363,6 +386,10 @@ public class KafkaNetworkChannelTest { code = ((FetchSnapshotResponseData) response).errorCode(); } else if (response instanceof UpdateRaftVoterResponseData) { code = ((UpdateRaftVoterResponseData) response).errorCode(); + } else if (response instanceof AddRaftVoterResponseData) { + code = ((AddRaftVoterResponseData) response).errorCode(); + } else if (response instanceof RemoveRaftVoterResponseData) { + code = ((RemoveRaftVoterResponseData) response).errorCode(); } else { throw new IllegalArgumentException("Unexpected type for responseData: " + response); } @@ -383,6 +410,10 @@ public class KafkaNetworkChannelTest { return new FetchSnapshotResponse((FetchSnapshotResponseData) responseData); } else if (responseData instanceof UpdateRaftVoterResponseData) { return new UpdateRaftVoterResponse((UpdateRaftVoterResponseData) responseData); + } else if (responseData instanceof AddRaftVoterResponseData) { + return new AddRaftVoterResponse((AddRaftVoterResponseData) responseData); + } else if (responseData instanceof RemoveRaftVoterResponseData) { + return new RemoveRaftVoterResponse((RemoveRaftVoterResponseData) responseData); } else { throw new IllegalArgumentException("Unexpected type for responseData: " + responseData); } diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java new file mode 100644 index 00000000000..7ad195e7d21 --- /dev/null +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientAutoJoinTest.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.raft; + +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.record.MemoryRecords; +import org.apache.kafka.common.utils.BufferSupplier; +import org.apache.kafka.server.common.KRaftVersion; + +import org.junit.jupiter.api.Test; + +import java.util.concurrent.ThreadLocalRandom; +import java.util.stream.Stream; + +import static org.apache.kafka.raft.KafkaRaftClientTest.replicaKey; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_595_PROTOCOL; +import static org.apache.kafka.raft.RaftClientTestContext.RaftProtocol.KIP_853_PROTOCOL; + +public class KafkaRaftClientAutoJoinTest { + @Test + public void testAutoRemoveOldVoter() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var oldFollower = replicaKey(leader.id() + 1, true); + final var newFollowerKey = replicaKey(oldFollower.id(), true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + newFollowerKey.id(), + newFollowerKey.directoryId().get() + ) + .withRaftProtocol(KIP_853_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, oldFollower)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(true) + .withCanBecomeVoter(true) + .build(); + + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the next request should be a remove voter request + pollAndDeliverRemoveVoter(context, oldFollower); + + // after sending a remove voter the next request should be a fetch + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the replica should send remove voter again because the fetch did not update the voter set + pollAndDeliverRemoveVoter(context, oldFollower); + } + + @Test + public void testAutoAddNewVoter() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var follower = replicaKey(leader.id() + 1, true); + final var newVoter = replicaKey(follower.id() + 1, true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + newVoter.id(), + newVoter.directoryId().get() + ) + .withRaftProtocol(KIP_853_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(true) + .withCanBecomeVoter(true) + .build(); + + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the next request should be an add voter request + pollAndSendAddVoter(context, newVoter); + + // expire the add voter request, the next request should be a fetch + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the replica should send add voter again because the completed fetch + // did not update the voter set, and its timer has expired + final var addVoterRequest = pollAndSendAddVoter(context, newVoter); + + // deliver the add voter response, this is possible before a completed fetch because of KIP-1186 + context.deliverResponse( + addVoterRequest.correlationId(), + addVoterRequest.destination(), + RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message()) + ); + + // verify the replica can perform a fetch to commit the new voter set + pollAndDeliverFetchToUpdateVoterSet( + context, + epoch, + VoterSetTest.voterSet(Stream.of(leader, newVoter)) + ); + } + + @Test + public void testObserverRemovesOldVoterAndAutoJoins() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var oldFollower = replicaKey(leader.id() + 1, true); + final var newFollowerKey = replicaKey(oldFollower.id(), true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + newFollowerKey.id(), + newFollowerKey.directoryId().get() + ) + .withRaftProtocol(KIP_853_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, oldFollower)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(true) + .withCanBecomeVoter(true) + .build(); + + // advance time and complete a fetch to trigger the remove voter request + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the next request should be a remove voter request + pollAndDeliverRemoveVoter(context, oldFollower); + + // after sending a remove voter the next request should be a fetch + // this fetch will remove the old follower from the voter set + pollAndDeliverFetchToUpdateVoterSet( + context, + epoch, + VoterSetTest.voterSet(Stream.of(leader)) + ); + + // advance time and complete a fetch to trigger the add voter request + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + // the next request should be an add voter request + final var addVoterRequest = pollAndSendAddVoter(context, newFollowerKey); + + // deliver the add voter response, this is possible before a completed fetch because of KIP-1186 + context.deliverResponse( + addVoterRequest.correlationId(), + addVoterRequest.destination(), + RaftUtil.addVoterResponse(Errors.NONE, Errors.NONE.message()) + ); + + // verify the replica can perform a fetch to commit the new voter set + pollAndDeliverFetchToUpdateVoterSet( + context, + epoch, + VoterSetTest.voterSet(Stream.of(leader, newFollowerKey)) + ); + + // advance time and complete a fetch and expire the update voter set timer + // the next request should be a fetch because the log voter configuration is up-to-date + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + context.pollUntilRequest(); + context.assertSentFetchRequest(); + } + + + @Test + public void testObserversDoNotAutoJoin() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var follower = replicaKey(leader.id() + 1, true); + final var newObserver = replicaKey(follower.id() + 1, true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + newObserver.id(), + newObserver.directoryId().get() + ) + .withRaftProtocol(KIP_853_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(true) + .withCanBecomeVoter(false) + .build(); + + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + + // When canBecomeVoter == false, the replica should not send an add voter request + final var fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + } + + @Test + public void testObserverDoesNotAddItselfWhenAutoJoinDisabled() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var follower = replicaKey(leader.id() + 1, true); + final var observer = replicaKey(follower.id() + 1, true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + observer.id(), + observer.directoryId().get() + ) + .withRaftProtocol(KIP_853_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_1 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(false) + .withCanBecomeVoter(true) + .build(); + + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + + // When autoJoin == false, the replica should not send an add voter request + final var fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + } + + @Test + public void testObserverDoesNotAutoJoinWithKRaftVersion0() throws Exception { + final var leader = replicaKey(randomReplicaId(), true); + final var follower = replicaKey(leader.id() + 1, true); + final var observer = replicaKey(follower.id() + 1, true); + final int epoch = 1; + final var context = new RaftClientTestContext.Builder( + observer.id(), + observer.directoryId().get() + ) + .withRaftProtocol(KIP_595_PROTOCOL) + .withStartingVoters( + VoterSetTest.voterSet(Stream.of(leader, follower)), KRaftVersion.KRAFT_VERSION_0 + ) + .withElectedLeader(epoch, leader.id()) + .withAutoJoin(true) + .withCanBecomeVoter(true) + .build(); + + context.advanceTimeAndCompleteFetch(epoch, leader.id(), true); + + context.time.sleep(context.fetchTimeoutMs - 1); + context.pollUntilRequest(); + + // When kraft.version == 0, the replica should not send an add voter request + final var fetchRequest = context.assertSentFetchRequest(); + + context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + } + + private void pollAndDeliverRemoveVoter( + RaftClientTestContext context, + ReplicaKey oldFollower + ) throws Exception { + context.pollUntilRequest(); + final var removeRequest = context.assertSentRemoveVoterRequest(oldFollower); + context.deliverResponse( + removeRequest.correlationId(), + removeRequest.destination(), + RaftUtil.removeVoterResponse(Errors.NONE, Errors.NONE.message()) + ); + } + + private RaftRequest.Outbound pollAndSendAddVoter( + RaftClientTestContext context, + ReplicaKey newVoter + ) throws Exception { + context.pollUntilRequest(); + return context.assertSentAddVoterRequest( + newVoter, + context.client.quorum().localVoterNodeOrThrow().listeners() + ); + } + + private void pollAndDeliverFetchToUpdateVoterSet( + RaftClientTestContext context, + int epoch, + VoterSet newVoterSet + ) throws Exception { + context.pollUntilRequest(); + final var fetchRequest = context.assertSentFetchRequest(); + context.assertFetchRequestData( + fetchRequest, + epoch, + context.log.endOffset().offset(), + context.log.lastFetchedEpoch(), + context.client.highWatermark() + ); + // deliver the fetch response with the updated voter set + context.deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + context.fetchResponse( + epoch, + fetchRequest.destination().id(), + MemoryRecords.withVotersRecord( + context.log.endOffset().offset(), + context.time.milliseconds(), + epoch, + BufferSupplier.NO_CACHING.get(300), + newVoterSet.toVotersRecord((short) 0) + ), + context.log.endOffset().offset() + 1, + Errors.NONE + ) + ); + // poll kraft to update the replica's voter set + context.client.poll(); + } + + private int randomReplicaId() { + return ThreadLocalRandom.current().nextInt(1025); + } +} diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java index 3bb8e93f5ea..dc70f7ce622 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientReconfigTest.java @@ -78,8 +78,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class KafkaRaftClientReconfigTest { - private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD = 1; - @Test public void testLeaderWritesBootstrapRecords() throws Exception { ReplicaKey local = replicaKey(randomReplicaId(), true); @@ -2225,28 +2223,8 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting for FETCH requests until the UpdateRaftVoter request is sent - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } - - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( local, @@ -2298,28 +2276,8 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting for FETCH request until the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } - - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( local, @@ -2389,28 +2347,8 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting for FETCH request until the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } - - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( local, @@ -2437,28 +2375,8 @@ public class KafkaRaftClientReconfigTest { context.pollUntilResponse(); // waiting for FETCH request until the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, newEpoch, 0L, 0, context.client.highWatermark()); + context.advanceTimeAndCompleteFetch(newEpoch, voter1.id(), true); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - newEpoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } - - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); updateRequest = context.assertSentUpdateVoterRequest( local, @@ -2723,29 +2641,9 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting for FETCH request until the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); - - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true); // update voter should not be sent because the local listener is not different from the voter set - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); @@ -2784,26 +2682,7 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting up to the last FETCH request before the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); - - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), false); // expect one last FETCH request context.pollUntilRequest(); @@ -2864,28 +2743,8 @@ public class KafkaRaftClientReconfigTest { .build(); // waiting for FETCH request until the UpdateRaftVoter request is set - for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_PERIOD; i++) { - context.time.sleep(context.fetchTimeoutMs - 1); - context.pollUntilRequest(); - RaftRequest.Outbound fetchRequest = context.assertSentFetchRequest(); - context.assertFetchRequestData(fetchRequest, epoch, 0L, 0, context.client.highWatermark()); + context.advanceTimeAndCompleteFetch(epoch, voter1.id(), true); - context.deliverResponse( - fetchRequest.correlationId(), - fetchRequest.destination(), - context.fetchResponse( - epoch, - voter1.id(), - MemoryRecords.EMPTY, - 0L, - Errors.NONE - ) - ); - // poll kraft to handle the fetch response - context.client.poll(); - } - - context.time.sleep(context.fetchTimeoutMs - 1); context.pollUntilRequest(); RaftRequest.Outbound updateRequest = context.assertSentUpdateVoterRequest( local, diff --git a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java index 4687fd3d903..1efd3247ebd 100644 --- a/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/KafkaRaftClientTest.java @@ -3786,7 +3786,7 @@ class KafkaRaftClientTest { @ParameterizedTest @CsvSource({ "true, true", "true, false", "false, true", "false, false" }) - public void testObserverReplication(boolean withKip853Rpc, boolean alwaysFlush) throws Exception { + public void testObserverReplication(boolean withKip853Rpc, boolean canBecomeVoter) throws Exception { int localId = randomReplicaId(); int otherNodeId = localId + 1; int epoch = 5; @@ -3795,7 +3795,7 @@ class KafkaRaftClientTest { RaftClientTestContext context = new RaftClientTestContext.Builder(localId, voters) .withElectedLeader(epoch, otherNodeId) .withKip853Rpc(withKip853Rpc) - .withAlwaysFlush(alwaysFlush) + .withCanBecomeVoter(canBecomeVoter) .build(); context.assertElectedLeader(epoch, otherNodeId); @@ -3812,7 +3812,7 @@ class KafkaRaftClientTest { context.client.poll(); assertEquals(2L, context.log.endOffset().offset()); - long firstUnflushedOffset = alwaysFlush ? 2L : 0L; + long firstUnflushedOffset = canBecomeVoter ? 2L : 0L; assertEquals(firstUnflushedOffset, context.log.firstUnflushedOffset()); } diff --git a/raft/src/test/java/org/apache/kafka/raft/QuorumConfigTest.java b/raft/src/test/java/org/apache/kafka/raft/QuorumConfigTest.java index 2197f3766c2..ce7175a8b5e 100644 --- a/raft/src/test/java/org/apache/kafka/raft/QuorumConfigTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/QuorumConfigTest.java @@ -34,6 +34,7 @@ public class QuorumConfigTest { assertInvalidConfig(Map.of(QuorumConfig.QUORUM_LINGER_MS_CONFIG, "-1")); assertInvalidConfig(Map.of(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, "-1")); assertInvalidConfig(Map.of(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, "-1")); + assertInvalidConfig(Map.of(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, "-1")); } private void assertInvalidConfig(Map overrideConfig) { @@ -46,6 +47,7 @@ public class QuorumConfigTest { props.put(QuorumConfig.QUORUM_LINGER_MS_CONFIG, "10"); props.put(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, "10"); props.put(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, "10"); + props.put(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, true); props.putAll(overrideConfig); diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java index 970f442c004..a98fb79d09a 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftClientTestContext.java @@ -143,10 +143,12 @@ public final class RaftClientTestContext { // Used to determine which RPC request and response to construct final RaftProtocol raftProtocol; // Used to determine if the local kraft client was configured to always flush - final boolean alwaysFlush; + final boolean canBecomeVoter; private final List sentResponses = new ArrayList<>(); + private static final int NUMBER_FETCH_TIMEOUTS_IN_UPDATE_VOTER_SET_PERIOD = 1; + public static final class Builder { static final int DEFAULT_ELECTION_TIMEOUT_MS = 10000; @@ -177,10 +179,11 @@ public final class RaftClientTestContext { private MemoryPool memoryPool = MemoryPool.NONE; private Optional> bootstrapServers = Optional.empty(); private RaftProtocol raftProtocol = RaftProtocol.KIP_595_PROTOCOL; - private boolean alwaysFlush = false; + private boolean canBecomeVoter = false; private VoterSet startingVoters = VoterSet.empty(); private Endpoints localListeners = Endpoints.empty(); private boolean isStartingVotersStatic = false; + private boolean autoJoin = false; public Builder(int localId, Set staticVoters) { this(OptionalInt.of(localId), staticVoters); @@ -309,8 +312,8 @@ public final class RaftClientTestContext { return this; } - Builder withAlwaysFlush(boolean alwaysFlush) { - this.alwaysFlush = alwaysFlush; + Builder withCanBecomeVoter(boolean canBecomeVoter) { + this.canBecomeVoter = canBecomeVoter; return this; } @@ -376,6 +379,11 @@ public final class RaftClientTestContext { return this; } + Builder withAutoJoin(boolean autoJoin) { + this.autoJoin = autoJoin; + return this; + } + public RaftClientTestContext build() throws IOException { Metrics metrics = new Metrics(time); MockNetworkChannel channel = new MockNetworkChannel(); @@ -404,13 +412,14 @@ public final class RaftClientTestContext { Endpoints.empty() : this.localListeners; - Map configMap = new HashMap<>(); + Map configMap = new HashMap<>(); configMap.put(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs); configMap.put(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, RETRY_BACKOFF_MS); configMap.put(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, electionTimeoutMs); configMap.put(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, ELECTION_BACKOFF_MAX_MS); configMap.put(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, FETCH_TIMEOUT_MS); configMap.put(QuorumConfig.QUORUM_LINGER_MS_CONFIG, appendLingerMs); + configMap.put(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG, autoJoin); QuorumConfig quorumConfig = new QuorumConfig(new AbstractConfig(QuorumConfig.CONFIG_DEF, configMap)); List computedBootstrapServers = bootstrapServers.orElseGet(() -> { @@ -436,7 +445,7 @@ public final class RaftClientTestContext { time, new MockExpirationService(time), FETCH_MAX_WAIT_MS, - alwaysFlush, + canBecomeVoter, clusterId, computedBootstrapServers, localListeners, @@ -474,7 +483,7 @@ public final class RaftClientTestContext { .boxed() .collect(Collectors.toSet()), raftProtocol, - alwaysFlush, + canBecomeVoter, metrics, externalKRaftMetrics, listener @@ -503,7 +512,7 @@ public final class RaftClientTestContext { VoterSet startingVoters, Set bootstrapIds, RaftProtocol raftProtocol, - boolean alwaysFlush, + boolean canBecomeVoter, Metrics metrics, ExternalKRaftMetrics externalKRaftMetrics, MockListener listener @@ -521,7 +530,7 @@ public final class RaftClientTestContext { this.startingVoters = startingVoters; this.bootstrapIds = bootstrapIds; this.raftProtocol = raftProtocol; - this.alwaysFlush = alwaysFlush; + this.canBecomeVoter = canBecomeVoter; this.metrics = metrics; this.externalKRaftMetrics = externalKRaftMetrics; this.listener = listener; @@ -949,6 +958,51 @@ public final class RaftClientTestContext { channel.mockReceive(new RaftResponse.Inbound(correlationId, versionedResponse, source)); } + /** + * Advance time and complete an empty fetch to reset the fetch timer. + * This is used to expire the update voter set timer without also expiring the fetch timer, + * which is needed for add, remove, and update voter tests. + * For voters and observers, polling after exiting this method expires the update voter set timer. + * @param epoch - the current epoch + * @param leaderId - the leader id + * @param expireUpdateVoterSetTimer - if true, advance time again to expire this timer + */ + void advanceTimeAndCompleteFetch( + int epoch, + int leaderId, + boolean expireUpdateVoterSetTimer + ) throws Exception { + for (int i = 0; i < NUMBER_FETCH_TIMEOUTS_IN_UPDATE_VOTER_SET_PERIOD; i++) { + time.sleep(fetchTimeoutMs - 1); + pollUntilRequest(); + final var fetchRequest = assertSentFetchRequest(); + assertFetchRequestData( + fetchRequest, + epoch, + log.endOffset().offset(), + log.lastFetchedEpoch(), + client.highWatermark() + ); + + deliverResponse( + fetchRequest.correlationId(), + fetchRequest.destination(), + fetchResponse( + epoch, + leaderId, + MemoryRecords.EMPTY, + log.endOffset().offset(), + Errors.NONE + ) + ); + // poll kraft to handle the fetch response + client.poll(); + } + if (expireUpdateVoterSetTimer) { + time.sleep(fetchTimeoutMs - 1); + } + } + List assertSentBeginQuorumEpochRequest(int epoch, Set destinationIds) { List requests = collectBeginEpochRequests(epoch); assertEquals(destinationIds.size(), requests.size()); @@ -1259,6 +1313,26 @@ public final class RaftClientTestContext { return sentRequests.get(0); } + RaftRequest.Outbound assertSentAddVoterRequest( + ReplicaKey replicaKey, + Endpoints endpoints + ) { + final var sentRequests = channel.drainSentRequests(Optional.of(ApiKeys.ADD_RAFT_VOTER)); + assertEquals(1, sentRequests.size()); + + final var request = sentRequests.get(0); + assertInstanceOf(AddRaftVoterRequestData.class, request.data()); + + final var addRaftVoterRequestData = (AddRaftVoterRequestData) request.data(); + assertEquals(clusterId, addRaftVoterRequestData.clusterId()); + assertEquals(replicaKey.id(), addRaftVoterRequestData.voterId()); + assertEquals(replicaKey.directoryId().get(), addRaftVoterRequestData.voterDirectoryId()); + assertEquals(endpoints, Endpoints.fromAddVoterRequest(addRaftVoterRequestData.listeners())); + assertEquals(false, addRaftVoterRequestData.ackWhenCommitted()); + + return request; + } + AddRaftVoterResponseData assertSentAddVoterResponse(Errors error) { List sentResponses = drainSentResponses(ApiKeys.ADD_RAFT_VOTER); assertEquals(1, sentResponses.size()); @@ -1272,6 +1346,23 @@ public final class RaftClientTestContext { return addVoterResponse; } + RaftRequest.Outbound assertSentRemoveVoterRequest( + ReplicaKey replicaKey + ) { + final var sentRequests = channel.drainSentRequests(Optional.of(ApiKeys.REMOVE_RAFT_VOTER)); + assertEquals(1, sentRequests.size()); + + final var request = sentRequests.get(0); + assertInstanceOf(RemoveRaftVoterRequestData.class, request.data()); + + final var removeRaftVoterRequestData = (RemoveRaftVoterRequestData) request.data(); + assertEquals(clusterId, removeRaftVoterRequestData.clusterId()); + assertEquals(replicaKey.id(), removeRaftVoterRequestData.voterId()); + assertEquals(replicaKey.directoryId().get(), removeRaftVoterRequestData.voterDirectoryId()); + + return request; + } + RemoveRaftVoterResponseData assertSentRemoveVoterResponse(Errors error) { List sentResponses = drainSentResponses(ApiKeys.REMOVE_RAFT_VOTER); assertEquals(1, sentResponses.size()); @@ -1707,7 +1798,7 @@ public final class RaftClientTestContext { // Assert that voters have flushed up to the fetch offset if ((localId.isPresent() && startingVoters.voterIds().contains(localId.getAsInt())) || - alwaysFlush + canBecomeVoter ) { assertEquals( log.firstUnflushedOffset(), @@ -1921,7 +2012,8 @@ public final class RaftClientTestContext { clusterId, timeoutMs, voter, - endpoints + endpoints, + true ); } diff --git a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java index 770a08b49ab..34b7c9f003d 100644 --- a/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java +++ b/raft/src/test/java/org/apache/kafka/raft/RaftUtilTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.compress.Compression; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.message.AddRaftVoterRequestData; import org.apache.kafka.common.message.AddRaftVoterRequestDataJsonConverter; +import org.apache.kafka.common.message.AddRaftVoterResponseData; import org.apache.kafka.common.message.BeginQuorumEpochRequestData; import org.apache.kafka.common.message.BeginQuorumEpochRequestDataJsonConverter; import org.apache.kafka.common.message.BeginQuorumEpochResponseData; @@ -42,6 +43,7 @@ import org.apache.kafka.common.message.FetchSnapshotRequestData; import org.apache.kafka.common.message.FetchSnapshotRequestDataJsonConverter; import org.apache.kafka.common.message.FetchSnapshotResponseData; import org.apache.kafka.common.message.FetchSnapshotResponseDataJsonConverter; +import org.apache.kafka.common.message.RemoveRaftVoterResponseData; import org.apache.kafka.common.message.VoteRequestData; import org.apache.kafka.common.message.VoteRequestDataJsonConverter; import org.apache.kafka.common.message.VoteResponseData; @@ -93,6 +95,10 @@ public class RaftUtilTest { RaftUtil.errorResponse(ApiKeys.FETCH, Errors.NONE)); assertEquals(new FetchSnapshotResponseData().setErrorCode(Errors.NONE.code()), RaftUtil.errorResponse(ApiKeys.FETCH_SNAPSHOT, Errors.NONE)); + assertEquals(new AddRaftVoterResponseData().setErrorCode(Errors.NONE.code()), + RaftUtil.errorResponse(ApiKeys.ADD_RAFT_VOTER, Errors.NONE)); + assertEquals(new RemoveRaftVoterResponseData().setErrorCode(Errors.NONE.code()), + RaftUtil.errorResponse(ApiKeys.REMOVE_RAFT_VOTER, Errors.NONE)); assertThrows(IllegalArgumentException.class, () -> RaftUtil.errorResponse(ApiKeys.PRODUCE, Errors.NONE)); } diff --git a/server/src/main/java/org/apache/kafka/server/share/fetch/InFlightState.java b/server/src/main/java/org/apache/kafka/server/share/fetch/InFlightState.java index d5831d74853..1dcdb52c90a 100644 --- a/server/src/main/java/org/apache/kafka/server/share/fetch/InFlightState.java +++ b/server/src/main/java/org/apache/kafka/server/share/fetch/InFlightState.java @@ -48,7 +48,7 @@ public class InFlightState { private String memberId; // The state of the records before the transition. In case we need to revert an in-flight state, we revert the above // attributes of InFlightState to this state, namely - state, deliveryCount and memberId. - private InFlightState rollbackState; + private RollbackState rollbackState; // The timer task for the acquisition lock timeout. private AcquisitionLockTimerTask acquisitionLockTimeoutTask; // The boolean determines if the record has achieved a terminal state of ARCHIVED from which it cannot transition @@ -205,7 +205,7 @@ public class InFlightState { InFlightState currentState = new InFlightState(state, deliveryCount, memberId, acquisitionLockTimeoutTask); InFlightState updatedState = tryUpdateState(newState, ops, maxDeliveryCount, newMemberId); if (updatedState != null) { - rollbackState = currentState; + rollbackState = new RollbackState(currentState, maxDeliveryCount); } return updatedState; } @@ -224,16 +224,23 @@ public class InFlightState { rollbackState = null; return; } + InFlightState previousState = rollbackState.state(); // Check is acquisition lock timeout task is expired then mark the message as Available. if (acquisitionLockTimeoutTask != null && acquisitionLockTimeoutTask.hasExpired()) { - state = RecordState.AVAILABLE; + // If the acquisition lock timeout task has expired, we should mark the record as available. + // However, if the delivery count has reached the maximum delivery count, we should archive the record. + state = previousState.deliveryCount() >= rollbackState.maxDeliveryCount ? + RecordState.ARCHIVED : RecordState.AVAILABLE; memberId = EMPTY_MEMBER_ID; cancelAndClearAcquisitionLockTimeoutTask(); } else { - state = rollbackState.state; - memberId = rollbackState.memberId; + state = previousState.state(); + memberId = previousState.memberId(); } - deliveryCount = rollbackState.deliveryCount(); + // Do not revert the delivery count as the delivery count should not be reverted for the failed + // state transition. However, in the current implementation, the delivery count is only incremented + // when the state is updated to Acquired, hence reverting the delivery count is not needed when + // the state transition fails. rollbackState = null; } @@ -271,4 +278,12 @@ public class InFlightState { ", memberId=" + memberId + ")"; } + + /** + * This record is used to store the state before the transition. It is used to revert the state if the transition fails. + * @param state The state of the records before the transition. + * @param maxDeliveryCount The maximum delivery count for the record. + */ + private record RollbackState(InFlightState state, int maxDeliveryCount) { + } } diff --git a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java index d9c7c91bb5c..a008971d399 100644 --- a/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java +++ b/streams/integration-tests/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.GroupProtocol; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -50,7 +51,8 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.Timeout; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.IOException; @@ -59,6 +61,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -71,6 +74,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.apache.kafka.streams.KafkaStreams.State.ERROR; import static org.apache.kafka.streams.KafkaStreams.State.REBALANCING; @@ -121,7 +125,7 @@ public class KStreamRepartitionIntegrationTest { CLUSTER.createTopic(outputTopic, 1, 1); } - private Properties createStreamsConfig(final String topologyOptimization) { + private Properties createStreamsConfig(final String topologyOptimization, final boolean useNewProtocol) { final Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); @@ -131,9 +135,23 @@ public class KStreamRepartitionIntegrationTest { streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsConfiguration.put(StreamsConfig.TOPOLOGY_OPTIMIZATION_CONFIG, topologyOptimization); + + if (useNewProtocol) { + streamsConfiguration.put(StreamsConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.STREAMS.name().toLowerCase(Locale.getDefault())); + } + return streamsConfiguration; } + private static Stream protocolAndOptimizationParameters() { + return Stream.of( + Arguments.of(StreamsConfig.OPTIMIZE, false), // OPTIMIZE with CLASSIC protocol + Arguments.of(StreamsConfig.OPTIMIZE, true), // OPTIMIZE with STREAMS protocol + Arguments.of(StreamsConfig.NO_OPTIMIZATION, false), // NO_OPTIMIZATION with CLASSIC protocol + Arguments.of(StreamsConfig.NO_OPTIMIZATION, true) // NO_OPTIMIZATION with STREAMS protocol + ); + } + @AfterEach public void whenShuttingDown() throws IOException { kafkaStreamsInstances.stream() @@ -144,8 +162,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining(final String topologyOptimization) throws InterruptedException { + @MethodSource("protocolAndOptimizationParameters") + public void shouldThrowAnExceptionWhenNumberOfPartitionsOfRepartitionOperationDoNotMatchSourceTopicWhenJoining(final String topologyOptimization, final boolean useNewProtocol) throws InterruptedException { final int topicBNumberOfPartitions = 6; final String inputTopicRepartitionName = "join-repartition-test"; final AtomicReference expectedThrowable = new AtomicReference<>(); @@ -167,10 +185,12 @@ public class KStreamRepartitionIntegrationTest { .join(topicBStream, (value1, value2) -> value2, JoinWindows.ofTimeDifferenceWithNoGrace(Duration.ofSeconds(10))) .to(outputTopic); - final Properties streamsConfiguration = createStreamsConfig(topologyOptimization); + final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol); try (final KafkaStreams ks = new KafkaStreams(builder.build(streamsConfiguration), streamsConfiguration)) { ks.setUncaughtExceptionHandler(exception -> { expectedThrowable.set(exception); + System.out.println(String.format("[%s Protocol] Exception caught: %s", + useNewProtocol ? "STREAMS" : "CLASSIC", exception.getMessage())); return SHUTDOWN_CLIENT; }); ks.start(); @@ -186,8 +206,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldDeductNumberOfPartitionsFromRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String topicBMapperName = "topic-b-mapper"; final int topicBNumberOfPartitions = 6; final String inputTopicRepartitionName = "join-repartition-test"; @@ -220,7 +240,7 @@ public class KStreamRepartitionIntegrationTest { .join(topicBStream, (value1, value2) -> value2, JoinWindows.of(Duration.ofSeconds(10))) .to(outputTopic); - final Properties streamsConfiguration = createStreamsConfig(topologyOptimization); + final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol); builder.build(streamsConfiguration); startStreams(builder, streamsConfiguration); @@ -239,8 +259,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldDoProperJoiningWhenNumberOfPartitionsAreValidWhenUsingRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String topicBRepartitionedName = "topic-b-scale-up"; final String inputTopicRepartitionedName = "input-topic-scale-up"; @@ -278,7 +298,7 @@ public class KStreamRepartitionIntegrationTest { .join(topicBStream, (value1, value2) -> value2, JoinWindows.of(Duration.ofSeconds(10))) .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); assertEquals(4, getNumberOfPartitionsForTopic(toRepartitionTopicName(topicBRepartitionedName))); assertEquals(4, getNumberOfPartitionsForTopic(toRepartitionTopicName(inputTopicRepartitionedName))); @@ -291,8 +311,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldRepartitionToMultiplePartitions(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldRepartitionToMultiplePartitions(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "broadcasting-partitioner-test"; final long timestamp = System.currentTimeMillis(); final AtomicInteger partitionerInvocation = new AtomicInteger(0); @@ -334,7 +354,7 @@ public class KStreamRepartitionIntegrationTest { .repartition(repartitioned) .to(broadcastingOutputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); final String topic = toRepartitionTopicName(repartitionName); @@ -360,8 +380,8 @@ public class KStreamRepartitionIntegrationTest { @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldUseStreamPartitionerForRepartitionOperation(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldUseStreamPartitionerForRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final int partition = 1; final String repartitionName = "partitioner-test"; final long timestamp = System.currentTimeMillis(); @@ -387,7 +407,7 @@ public class KStreamRepartitionIntegrationTest { .repartition(repartitioned) .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); final String topic = toRepartitionTopicName(repartitionName); @@ -402,8 +422,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldPerformSelectKeyWithRepartitionOperation(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldPerformSelectKeyWithRepartitionOperation(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final long timestamp = System.currentTimeMillis(); sendEvents( @@ -421,7 +441,7 @@ public class KStreamRepartitionIntegrationTest { .repartition() .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new IntegerDeserializer(), @@ -438,8 +458,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldCreateRepartitionTopicIfKeyChangingOperationWasNotPerformed(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "dummy"; final long timestamp = System.currentTimeMillis(); @@ -457,7 +477,7 @@ public class KStreamRepartitionIntegrationTest { .repartition(Repartitioned.as(repartitionName)) .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new IntegerDeserializer(), @@ -475,8 +495,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldPerformKeySelectOperationWhenRepartitionOperationIsUsedWithKeySelector(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionedName = "new-key"; final long timestamp = System.currentTimeMillis(); @@ -501,7 +521,7 @@ public class KStreamRepartitionIntegrationTest { .toStream() .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new StringDeserializer(), @@ -521,8 +541,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldCreateRepartitionTopicWithSpecifiedNumberOfPartitions(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "new-partitions"; final long timestamp = System.currentTimeMillis(); @@ -543,7 +563,7 @@ public class KStreamRepartitionIntegrationTest { .toStream() .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new IntegerDeserializer(), @@ -561,8 +581,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldInheritRepartitionTopicPartitionNumberFromUpstreamTopicWhenNumberOfPartitionsIsNotSpecified(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "new-topic"; final long timestamp = System.currentTimeMillis(); @@ -583,7 +603,7 @@ public class KStreamRepartitionIntegrationTest { .toStream() .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new IntegerDeserializer(), @@ -601,8 +621,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldCreateOnlyOneRepartitionTopicWhenRepartitionIsFollowedByGroupByKey(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "new-partitions"; final long timestamp = System.currentTimeMillis(); @@ -629,7 +649,7 @@ public class KStreamRepartitionIntegrationTest { .toStream() .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); final String topology = builder.build().describe().toString(); @@ -647,8 +667,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldGenerateRepartitionTopicWhenNameIsNotSpecified(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final long timestamp = System.currentTimeMillis(); sendEvents( @@ -666,7 +686,7 @@ public class KStreamRepartitionIntegrationTest { .repartition(Repartitioned.with(Serdes.String(), Serdes.String())) .to(outputTopic); - startStreams(builder, createStreamsConfig(topologyOptimization)); + startStreams(builder, createStreamsConfig(topologyOptimization, useNewProtocol)); validateReceivedMessages( new StringDeserializer(), @@ -683,8 +703,8 @@ public class KStreamRepartitionIntegrationTest { } @ParameterizedTest - @ValueSource(strings = {StreamsConfig.OPTIMIZE, StreamsConfig.NO_OPTIMIZATION}) - public void shouldGoThroughRebalancingCorrectly(final String topologyOptimization) throws Exception { + @MethodSource("protocolAndOptimizationParameters") + public void shouldGoThroughRebalancingCorrectly(final String topologyOptimization, final boolean useNewProtocol) throws Exception { final String repartitionName = "rebalancing-test"; final long timestamp = System.currentTimeMillis(); @@ -711,7 +731,7 @@ public class KStreamRepartitionIntegrationTest { .toStream() .to(outputTopic); - final Properties streamsConfiguration = createStreamsConfig(topologyOptimization); + final Properties streamsConfiguration = createStreamsConfig(topologyOptimization, useNewProtocol); startStreams(builder, streamsConfiguration); final Properties streamsToCloseConfigs = new Properties(); streamsToCloseConfigs.putAll(streamsConfiguration); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 65dec15b3ee..d8932206217 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -45,6 +45,7 @@ import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Timer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.GroupProtocol; import org.apache.kafka.streams.KafkaClientSupplier; @@ -56,6 +57,7 @@ import org.apache.kafka.streams.errors.MissingSourceTopicException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; +import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.internals.ConsumerWrapper; import org.apache.kafka.streams.internals.metrics.ClientMetrics; import org.apache.kafka.streams.internals.metrics.StreamsThreadMetricsDelegatingReporter; @@ -372,6 +374,8 @@ public class StreamThread extends Thread implements ProcessingThread { private volatile KafkaFutureImpl restoreConsumerInstanceIdFuture = new KafkaFutureImpl<>(); private volatile KafkaFutureImpl producerInstanceIdFuture = new KafkaFutureImpl<>(); + private Timer topicsReadyTimer; + public static StreamThread create(final TopologyMetadata topologyMetadata, final StreamsConfig config, final KafkaClientSupplier clientSupplier, @@ -1536,15 +1540,28 @@ public class StreamThread extends Thread implements ProcessingThread { public void handleStreamsRebalanceData() { if (streamsRebalanceData.isPresent()) { + boolean hasMissingSourceTopics = false; + String missingTopicsDetail = null; + for (final StreamsGroupHeartbeatResponseData.Status status : streamsRebalanceData.get().statuses()) { if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.SHUTDOWN_APPLICATION.code()) { shutdownErrorHook.run(); } else if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) { - final String errorMsg = String.format("Missing source topics: %s", status.statusDetail()); + hasMissingSourceTopics = true; + missingTopicsDetail = status.statusDetail(); + } else if (status.statusCode() == StreamsGroupHeartbeatResponse.Status.INCORRECTLY_PARTITIONED_TOPICS.code()) { + final String errorMsg = status.statusDetail(); log.error(errorMsg); - throw new MissingSourceTopicException(errorMsg); + throw new TopologyException(errorMsg); } } + + if (hasMissingSourceTopics) { + handleMissingSourceTopicsWithTimeout(missingTopicsDetail); + } else { + // Reset timeout tracking when no missing source topics are reported + topicsReadyTimer = null; + } final Map partitionsByEndpoint = streamsRebalanceData.get().partitionsByHost(); @@ -1563,6 +1580,30 @@ public class StreamThread extends Thread implements ProcessingThread { } } + private void handleMissingSourceTopicsWithTimeout(final String missingTopicsDetail) { + // Start timeout tracking on first encounter with missing topics + if (topicsReadyTimer == null) { + topicsReadyTimer = time.timer(maxPollTimeMs); + log.info("Missing source topics detected: {}. Will wait up to {}ms before failing.", + missingTopicsDetail, maxPollTimeMs); + } else { + topicsReadyTimer.update(); + } + + if (topicsReadyTimer.isExpired()) { + final long elapsedTime = topicsReadyTimer.elapsedMs(); + final String errorMsg = String.format("Missing source topics: %s. Timeout exceeded after %dms.", + missingTopicsDetail, elapsedTime); + log.error(errorMsg); + + throw new MissingSourceTopicException(errorMsg); + } else { + log.debug("Missing source topics: {}. Elapsed time: {}ms, timeout in: {}ms", + missingTopicsDetail, topicsReadyTimer.elapsedMs(), topicsReadyTimer.remainingMs()); + } + } + + static Map getTopicPartitionInfo(final Map> partitionsByHost) { final Map topicToPartitionInfo = new HashMap<>(); for (final Set value : partitionsByHost.values()) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index a6ab8d8209a..213ff8a5b10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -70,6 +70,7 @@ import org.apache.kafka.streams.errors.MissingSourceTopicException; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.errors.TaskCorruptedException; import org.apache.kafka.streams.errors.TaskMigratedException; +import org.apache.kafka.streams.errors.TopologyException; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.internals.ConsumedInternal; @@ -162,6 +163,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isA; import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertInstanceOf; @@ -3896,8 +3898,9 @@ public class StreamThreadTest { new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) ); final StreamsConfig config = new StreamsConfig(props); + final MockTime mockTime = new MockTime(1); thread = new StreamThread( - new MockTime(1), + mockTime, config, null, mainConsumer, @@ -3929,8 +3932,77 @@ public class StreamThreadTest { .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) .setStatusDetail("Missing source topics") )); + + // First call should not throw exception (within timeout) + thread.runOnceWithoutProcessingThreads(); + + // Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout + mockTime.sleep(300001); + final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithoutProcessingThreads()); - assertTrue(exception.getMessage().startsWith("Missing source topics")); + assertTrue(exception.getMessage().contains("Missing source topics")); + assertTrue(exception.getMessage().contains("Timeout exceeded")); + } + + @Test + public void testStreamsProtocolIncorrectlyPartitionedTopics() { + final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class); + when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); + when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of())); + when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata); + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of(), + Map.of() + ); + final Runnable shutdownErrorHook = mock(Runnable.class); + + final Properties props = configProps(false, false, false); + final StreamsMetadataState streamsMetadataState = new StreamsMetadataState( + new TopologyMetadata(internalTopologyBuilder, new StreamsConfig(props)), + StreamsMetadataState.UNKNOWN_HOST, + new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) + ); + final StreamsConfig config = new StreamsConfig(props); + final MockTime mockTime = new MockTime(1); + thread = new StreamThread( + mockTime, + config, + null, + mainConsumer, + consumer, + changelogReader, + null, + mock(TaskManager.class), + null, + new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime), + new TopologyMetadata(internalTopologyBuilder, config), + PROCESS_ID, + CLIENT_ID, + new LogContext(""), + null, + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + shutdownErrorHook, + HANDLER, + null, + Optional.of(streamsRebalanceData), + streamsMetadataState + ).updateThreadMetadata(adminClientId(CLIENT_ID)); + + thread.setState(State.STARTING); + + streamsRebalanceData.setStatuses(List.of( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.INCORRECTLY_PARTITIONED_TOPICS.code()) + .setStatusDetail("Topics are incorrectly partitioned") + )); + + // Should immediately throw TopologyException (no timeout like MISSING_SOURCE_TOPICS) + final TopologyException exception = assertThrows(TopologyException.class, + () -> thread.runOnceWithoutProcessingThreads()); + assertTrue(exception.getMessage().contains("Topics are incorrectly partitioned")); } @Test @@ -4013,8 +4085,9 @@ public class StreamThreadTest { StreamsMetadataState.UNKNOWN_HOST, new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) ); + final MockTime mockTime = new MockTime(1); thread = new StreamThread( - new MockTime(1), + mockTime, config, null, mainConsumer, @@ -4046,8 +4119,105 @@ public class StreamThreadTest { .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) .setStatusDetail("Missing source topics") )); + + // First call should not throw exception (within timeout) + thread.runOnceWithProcessingThreads(); + + // Advance time beyond max.poll.interval.ms (default is 300000ms) to trigger timeout + mockTime.sleep(300001); + final MissingSourceTopicException exception = assertThrows(MissingSourceTopicException.class, () -> thread.runOnceWithProcessingThreads()); - assertTrue(exception.getMessage().startsWith("Missing source topics")); + assertTrue(exception.getMessage().contains("Missing source topics")); + assertTrue(exception.getMessage().contains("Timeout exceeded")); + } + + @Test + public void testStreamsProtocolMissingSourceTopicRecovery() { + final ConsumerGroupMetadata consumerGroupMetadata = Mockito.mock(ConsumerGroupMetadata.class); + when(consumerGroupMetadata.groupInstanceId()).thenReturn(Optional.empty()); + when(mainConsumer.poll(Mockito.any(Duration.class))).thenReturn(new ConsumerRecords<>(Map.of(), Map.of())); + when(mainConsumer.groupMetadata()).thenReturn(consumerGroupMetadata); + final StreamsRebalanceData streamsRebalanceData = new StreamsRebalanceData( + UUID.randomUUID(), + Optional.empty(), + Map.of(), + Map.of() + ); + + final Properties props = configProps(false, false, false); + final Runnable shutdownErrorHook = mock(Runnable.class); + final StreamsConfig config = new StreamsConfig(props); + final StreamsMetadataState streamsMetadataState = new StreamsMetadataState( + new TopologyMetadata(internalTopologyBuilder, config), + StreamsMetadataState.UNKNOWN_HOST, + new LogContext(String.format("stream-client [%s] ", CLIENT_ID)) + ); + final MockTime mockTime = new MockTime(1); + thread = new StreamThread( + mockTime, + config, + null, + mainConsumer, + consumer, + changelogReader, + null, + mock(TaskManager.class), + null, + new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime), + new TopologyMetadata(internalTopologyBuilder, config), + PROCESS_ID, + CLIENT_ID, + new LogContext(""), + null, + new AtomicLong(Long.MAX_VALUE), + new LinkedList<>(), + shutdownErrorHook, + HANDLER, + null, + Optional.of(streamsRebalanceData), + streamsMetadataState + ).updateThreadMetadata(adminClientId(CLIENT_ID)); + + thread.setState(State.STARTING); + thread.runOnceWithoutProcessingThreads(); + + // Set missing source topics status + streamsRebalanceData.setStatuses(List.of( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) + .setStatusDetail("Missing source topics") + )); + + // First call should not throw exception (within timeout) + thread.runOnceWithoutProcessingThreads(); + + // Advance time but not beyond timeout + mockTime.sleep(150000); // Half of max.poll.interval.ms + + // Should still not throw exception + thread.runOnceWithoutProcessingThreads(); + + // Clear the missing source topics (simulate recovery) + streamsRebalanceData.setStatuses(List.of()); + + // Should complete without exception (recovery successful) + assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads()); + + // Set missing topics again - should reset the timeout + streamsRebalanceData.setStatuses(List.of( + new StreamsGroupHeartbeatResponseData.Status() + .setStatusCode(StreamsGroupHeartbeatResponse.Status.MISSING_SOURCE_TOPICS.code()) + .setStatusDetail("Different missing topics") + )); + + // Advance time by 250 seconds to test if timer was reset + // Total time from beginning: 150000 + 250000 = 400000ms (400s) + // If timer was NOT reset: elapsed time = 400s > 300s → should throw + // If timer WAS reset: elapsed time = 250s < 300s → should NOT throw + mockTime.sleep(250000); // Advance by 250 seconds + + // Should not throw because timer was reset - only 250s elapsed from reset point + assertDoesNotThrow(() -> thread.runOnceWithoutProcessingThreads()); } @Test diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java index 45b7cada936..59041c7a66a 100644 --- a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java @@ -27,10 +27,12 @@ import kafka.server.SharedServer; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.common.Uuid; import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.test.api.TestKitDefaults; import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; @@ -114,6 +116,8 @@ public class KafkaClusterTestKit implements AutoCloseable { private final String controllerListenerName; private final String brokerSecurityProtocol; private final String controllerSecurityProtocol; + private boolean standalone; + private Optional> initialVoterSet = Optional.empty(); private boolean deleteOnClose; public Builder(TestKitNodes nodes) { @@ -130,6 +134,16 @@ public class KafkaClusterTestKit implements AutoCloseable { return this; } + public Builder setStandalone(boolean standalone) { + this.standalone = standalone; + return this; + } + + public Builder setInitialVoterSet(Map initialVoterSet) { + this.initialVoterSet = Optional.of(initialVoterSet); + return this; + } + private KafkaConfig createNodeConfig(TestKitNode node) throws IOException { TestKitNode brokerNode = nodes.brokerNodes().get(node.id()); TestKitNode controllerNode = nodes.controllerNodes().get(node.id()); @@ -184,6 +198,11 @@ public class KafkaClusterTestKit implements AutoCloseable { // reduce log cleaner offset map memory usage props.putIfAbsent(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152"); + // do not include auto join config in broker nodes + if (brokerNode != null) { + props.remove(QuorumConfig.QUORUM_AUTO_JOIN_ENABLE_CONFIG); + } + // Add associated broker node property overrides if (brokerNode != null) { props.putAll(brokerNode.propertyOverrides()); @@ -323,6 +342,8 @@ public class KafkaClusterTestKit implements AutoCloseable { faultHandlerFactory, socketFactoryManager, jaasFile, + standalone, + initialVoterSet, deleteOnClose); } @@ -368,6 +389,8 @@ public class KafkaClusterTestKit implements AutoCloseable { private final PreboundSocketFactoryManager socketFactoryManager; private final String controllerListenerName; private final Optional jaasFile; + private final boolean standalone; + private final Optional> initialVoterSet; private final boolean deleteOnClose; private KafkaClusterTestKit( @@ -378,6 +401,8 @@ public class KafkaClusterTestKit implements AutoCloseable { SimpleFaultHandlerFactory faultHandlerFactory, PreboundSocketFactoryManager socketFactoryManager, Optional jaasFile, + boolean standalone, + Optional> initialVoterSet, boolean deleteOnClose ) { /* @@ -395,6 +420,8 @@ public class KafkaClusterTestKit implements AutoCloseable { this.socketFactoryManager = socketFactoryManager; this.controllerListenerName = nodes.controllerListenerName().value(); this.jaasFile = jaasFile; + this.standalone = standalone; + this.initialVoterSet = initialVoterSet; this.deleteOnClose = deleteOnClose; } @@ -425,8 +452,9 @@ public class KafkaClusterTestKit implements AutoCloseable { boolean writeMetadataDirectory ) { try { + final var nodeId = ensemble.nodeId().getAsInt(); Formatter formatter = new Formatter(); - formatter.setNodeId(ensemble.nodeId().getAsInt()); + formatter.setNodeId(nodeId); formatter.setClusterId(ensemble.clusterId().get()); if (writeMetadataDirectory) { formatter.setDirectories(ensemble.logDirProps().keySet()); @@ -452,15 +480,50 @@ public class KafkaClusterTestKit implements AutoCloseable { if (nodes.bootstrapMetadata().featureLevel(KRaftVersion.FEATURE_NAME) > 0) { StringBuilder dynamicVotersBuilder = new StringBuilder(); String prefix = ""; - for (TestKitNode controllerNode : nodes.controllerNodes().values()) { - int port = socketFactoryManager. - getOrCreatePortForListener(controllerNode.id(), controllerListenerName); - dynamicVotersBuilder.append(prefix); - prefix = ","; - dynamicVotersBuilder.append(String.format("%d@localhost:%d:%s", - controllerNode.id(), port, controllerNode.metadataDirectoryId())); + if (standalone) { + if (nodeId == TestKitDefaults.CONTROLLER_ID_OFFSET) { + final var controllerNode = nodes.controllerNodes().get(nodeId); + dynamicVotersBuilder.append( + String.format( + "%d@localhost:%d:%s", + controllerNode.id(), + socketFactoryManager. + getOrCreatePortForListener(controllerNode.id(), controllerListenerName), + controllerNode.metadataDirectoryId() + ) + ); + formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); + } else { + formatter.setNoInitialControllersFlag(true); + } + } else if (initialVoterSet.isPresent()) { + for (final var controllerNode : initialVoterSet.get().entrySet()) { + final var voterId = controllerNode.getKey(); + final var voterDirectoryId = controllerNode.getValue(); + dynamicVotersBuilder.append(prefix); + prefix = ","; + dynamicVotersBuilder.append( + String.format( + "%d@localhost:%d:%s", + voterId, + socketFactoryManager. + getOrCreatePortForListener(voterId, controllerListenerName), + voterDirectoryId + ) + ); + } + formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); + } else { + for (TestKitNode controllerNode : nodes.controllerNodes().values()) { + int port = socketFactoryManager. + getOrCreatePortForListener(controllerNode.id(), controllerListenerName); + dynamicVotersBuilder.append(prefix); + prefix = ","; + dynamicVotersBuilder.append(String.format("%d@localhost:%d:%s", + controllerNode.id(), port, controllerNode.metadataDirectoryId())); + } + formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); } - formatter.setInitialControllers(DynamicVoters.parse(dynamicVotersBuilder.toString())); } formatter.run(); } catch (Exception e) { diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/PartitionReassignmentState.java b/tools/src/main/java/org/apache/kafka/tools/reassign/PartitionReassignmentState.java index 53ae960f863..a81b430c8b3 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/PartitionReassignmentState.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/PartitionReassignmentState.java @@ -18,40 +18,17 @@ package org.apache.kafka.tools.reassign; import java.util.List; -import java.util.Objects; /** * The state of a partition reassignment. The current replicas and target replicas * may overlap. + * + * @param currentReplicas The current replicas. + * @param targetReplicas The target replicas. + * @param done True if the reassignment is done. */ -final class PartitionReassignmentState { - public final List currentReplicas; - - public final List targetReplicas; - - public final boolean done; - - /** - * @param currentReplicas The current replicas. - * @param targetReplicas The target replicas. - * @param done True if the reassignment is done. - */ - public PartitionReassignmentState(List currentReplicas, List targetReplicas, boolean done) { - this.currentReplicas = currentReplicas; - this.targetReplicas = targetReplicas; - this.done = done; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - PartitionReassignmentState state = (PartitionReassignmentState) o; - return done == state.done && Objects.equals(currentReplicas, state.currentReplicas) && Objects.equals(targetReplicas, state.targetReplicas); - } - - @Override - public int hashCode() { - return Objects.hash(currentReplicas, targetReplicas, done); - } -} +record PartitionReassignmentState( + List currentReplicas, + List targetReplicas, + boolean done +) { } diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java index 334f0738ca3..591c127ae2f 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommand.java @@ -279,12 +279,12 @@ public class ReassignPartitionsCommand { bld.add("Status of partition reassignment:"); states.keySet().stream().sorted(ReassignPartitionsCommand::compareTopicPartitions).forEach(topicPartition -> { PartitionReassignmentState state = states.get(topicPartition); - if (state.done) { - if (state.currentReplicas.equals(state.targetReplicas)) { + if (state.done()) { + if (state.currentReplicas().equals(state.targetReplicas())) { bld.add(String.format("Reassignment of partition %s is completed.", topicPartition)); } else { - String currentReplicaStr = state.currentReplicas.stream().map(String::valueOf).collect(Collectors.joining(",")); - String targetReplicaStr = state.targetReplicas.stream().map(String::valueOf).collect(Collectors.joining(",")); + String currentReplicaStr = state.currentReplicas().stream().map(String::valueOf).collect(Collectors.joining(",")); + String targetReplicaStr = state.targetReplicas().stream().map(String::valueOf).collect(Collectors.joining(",")); bld.add("There is no active reassignment of partition " + topicPartition + ", " + "but replica set is " + currentReplicaStr + " rather than " + diff --git a/tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java b/tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java index 9dbde0fd1af..0232eb3d1c1 100644 --- a/tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java +++ b/tools/src/main/java/org/apache/kafka/tools/reassign/VerifyAssignmentResult.java @@ -21,49 +21,21 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartitionReplica; import java.util.Map; -import java.util.Objects; /** * A result returned from verifyAssignment. + * @param partStates A map from partitions to reassignment states. + * @param partsOngoing True if there are any ongoing partition reassignments. + * @param moveStates A map from log directories to movement states. + * @param movesOngoing True if there are any ongoing moves that we know about. */ -public final class VerifyAssignmentResult { - public final Map partStates; - public final boolean partsOngoing; - public final Map moveStates; - public final boolean movesOngoing; - +public record VerifyAssignmentResult( + Map partStates, + boolean partsOngoing, + Map moveStates, + boolean movesOngoing +) { public VerifyAssignmentResult(Map partStates) { this(partStates, false, Map.of(), false); } - - /** - * @param partStates A map from partitions to reassignment states. - * @param partsOngoing True if there are any ongoing partition reassignments. - * @param moveStates A map from log directories to movement states. - * @param movesOngoing True if there are any ongoing moves that we know about. - */ - public VerifyAssignmentResult( - Map partStates, - boolean partsOngoing, - Map moveStates, - boolean movesOngoing - ) { - this.partStates = partStates; - this.partsOngoing = partsOngoing; - this.moveStates = moveStates; - this.movesOngoing = movesOngoing; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - VerifyAssignmentResult that = (VerifyAssignmentResult) o; - return partsOngoing == that.partsOngoing && movesOngoing == that.movesOngoing && Objects.equals(partStates, that.partStates) && Objects.equals(moveStates, that.moveStates); - } - - @Override - public int hashCode() { - return Objects.hash(partStates, partsOngoing, moveStates, movesOngoing); - } } diff --git a/tools/src/test/java/org/apache/kafka/tools/AbstractResetIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/AbstractResetIntegrationTest.java index dbfba1ad241..57c4a5e8bcb 100644 --- a/tools/src/test/java/org/apache/kafka/tools/AbstractResetIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/AbstractResetIntegrationTest.java @@ -51,12 +51,10 @@ import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.stream.Collectors; +import java.util.Set; import static java.time.Duration.ofMillis; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForEmptyConsumerGroup; @@ -176,7 +174,7 @@ public abstract class AbstractResetIntegrationTest { } private void add10InputElements() { - final List> records = Arrays.asList(KeyValue.pair(0L, "aaa"), + final List> records = List.of(KeyValue.pair(0L, "aaa"), KeyValue.pair(1L, "bbb"), KeyValue.pair(0L, "ccc"), KeyValue.pair(1L, "ddd"), @@ -189,7 +187,7 @@ public abstract class AbstractResetIntegrationTest { for (final KeyValue record : records) { mockTime.sleep(10); - IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Collections.singleton(record), producerConfig, mockTime.milliseconds()); + IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(INPUT_TOPIC, Set.of(record), producerConfig, mockTime.milliseconds()); } } @@ -211,7 +209,7 @@ public abstract class AbstractResetIntegrationTest { final List internalTopics = cluster.getAllTopicsInCluster().stream() .filter(StreamsResetter::matchesInternalTopicFormat) - .collect(Collectors.toList()); + .toList(); cleanGlobal(false, "--internal-topics", String.join(",", internalTopics.subList(1, internalTopics.size())), @@ -288,7 +286,7 @@ public abstract class AbstractResetIntegrationTest { if (!useRepartitioned) { IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp( INTERMEDIATE_USER_TOPIC, - Collections.singleton(badMessage), + Set.of(badMessage), producerConfig, mockTime.milliseconds()); } @@ -375,7 +373,7 @@ public abstract class AbstractResetIntegrationTest { final String resetScenarioArg, final String appID) throws Exception { final List parameterList = new ArrayList<>( - Arrays.asList("--application-id", appID, + List.of("--application-id", appID, "--bootstrap-server", cluster.bootstrapServers(), "--input-topics", INPUT_TOPIC )); diff --git a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java index b1cc54c828a..cfc9b686644 100644 --- a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java @@ -31,8 +31,8 @@ import org.apache.kafka.common.test.api.ClusterTestDefaults; import org.apache.kafka.server.config.ServerConfigs; import java.util.Arrays; -import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Properties; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -56,7 +56,7 @@ public class BrokerApiVersionsCommandTest { NodeApiVersions nodeApiVersions = new NodeApiVersions( ApiVersionsResponse.filterApis(listenerType, true, true), - Collections.emptyList()); + List.of()); Iterator apiKeysIter = ApiKeys.clientApis().iterator(); while (apiKeysIter.hasNext()) { ApiKeys apiKey = apiKeysIter.next(); diff --git a/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java index 78ef5542372..b389e114332 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ClientMetricsCommandTest.java @@ -33,7 +33,6 @@ import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import java.util.Collection; -import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -209,7 +208,7 @@ public class ClientMetricsCommandTest { ClientMetricsCommand.ClientMetricsService service = new ClientMetricsCommand.ClientMetricsService(adminClient); ConfigResource cr = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, clientMetricsName); - Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer."))); + Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer."))); DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg); when(adminClient.describeConfigs(any())).thenReturn(describeResult); AlterConfigsResult alterResult = AdminClientTestUtils.alterConfigsResult(cr); @@ -237,7 +236,7 @@ public class ClientMetricsCommandTest { ConfigResource.Type.CLIENT_METRICS, Set.of(clientMetricsName) )); when(adminClient.listConfigResources(any(), any())).thenReturn(listConfigResourcesResult); - Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer."))); + Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer."))); DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg); when(adminClient.describeConfigs(any())).thenReturn(describeResult); @@ -284,7 +283,7 @@ public class ClientMetricsCommandTest { ListConfigResourcesResult result = AdminClientTestUtils.listConfigResourcesResult(clientMetricsName); when(adminClient.listConfigResources(any(), any())).thenReturn(result); ConfigResource cr = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, clientMetricsName); - Config cfg = new Config(Collections.singleton(new ConfigEntry("metrics", "org.apache.kafka.producer."))); + Config cfg = new Config(Set.of(new ConfigEntry("metrics", "org.apache.kafka.producer."))); DescribeConfigsResult describeResult = AdminClientTestUtils.describeConfigsResult(cr, cfg); when(adminClient.describeConfigs(any())).thenReturn(describeResult); @@ -326,7 +325,7 @@ public class ClientMetricsCommandTest { ListConfigResourcesResult result = AdminClientTestUtils.listConfigResourcesResult(Errors.UNSUPPORTED_VERSION.exception()); when(adminClient.listConfigResources(any(), any())).thenReturn(result); - assertThrows(ExecutionException.class, () -> service.listClientMetrics()); + assertThrows(ExecutionException.class, service::listClientMetrics); } private void assertInitializeInvalidOptionsExitCode(int expected, String[] options) { diff --git a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java index 6a2fcc5150e..201f8e56f4e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java @@ -29,7 +29,6 @@ import java.io.PrintStream; import java.util.Arrays; import java.util.List; import java.util.Set; -import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -71,10 +70,10 @@ public class ClusterToolTest { @ClusterTest(brokers = 2, types = {Type.KRAFT, Type.CO_KRAFT}) public void testListEndpointsArgumentWithBootstrapServer(ClusterInstance clusterInstance) { - List brokerIds = clusterInstance.brokerIds().stream().collect(Collectors.toList()); + List brokerIds = clusterInstance.brokerIds().stream().toList(); clusterInstance.shutdownBroker(brokerIds.get(0)); - List ports = Arrays.stream(clusterInstance.bootstrapServers().split(",")).map(b -> b.split(":")[1]).collect(Collectors.toList()); + List ports = Arrays.stream(clusterInstance.bootstrapServers().split(",")).map(b -> b.split(":")[1]).toList(); String format = "%-10s %-9s %-10s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s %-6s"; String expected = String.format(format, "ID", "HOST", "PORT", "RACK", "STATE", "ENDPOINT_TYPE", @@ -110,7 +109,7 @@ public class ClusterToolTest { int id = clusterInstance.controllerIds().iterator().next(); String format = "%-10s %-9s %-10s %-10s %-15s%n%-10s %-9s %-10s %-10s %-10s"; String expected = String.format(format, "ID", "HOST", "PORT", "RACK", "ENDPOINT_TYPE", id, "localhost", port, "null", "controller"); - assertTrue(output.equals(expected)); + assertEquals(expected, output); } @ClusterTest(brokers = 3, types = {Type.KRAFT, Type.CO_KRAFT}) diff --git a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java index c494821a5b0..bdaa6ee5934 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandIntegrationTest.java @@ -52,10 +52,6 @@ import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static java.util.Collections.singletonMap; import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG; import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG; import static org.apache.kafka.common.config.SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG; @@ -246,27 +242,27 @@ public class ConfigCommandIntegrationTest { try (Admin client = cluster.admin()) { // Add config - alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "110000"), alterOpts); - alterAndVerifyConfig(client, Optional.empty(), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "120000"), alterOpts); + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), Map.of(MESSAGE_MAX_BYTES_CONFIG, "110000"), alterOpts); + alterAndVerifyConfig(client, Optional.empty(), Map.of(MESSAGE_MAX_BYTES_CONFIG, "120000"), alterOpts); // Change config - alterAndVerifyConfig(client, Optional.of(defaultBrokerId), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "130000"), alterOpts); - alterAndVerifyConfig(client, Optional.empty(), singletonMap(MESSAGE_MAX_BYTES_CONFIG, "140000"), alterOpts); + alterAndVerifyConfig(client, Optional.of(defaultBrokerId), Map.of(MESSAGE_MAX_BYTES_CONFIG, "130000"), alterOpts); + alterAndVerifyConfig(client, Optional.empty(), Map.of(MESSAGE_MAX_BYTES_CONFIG, "140000"), alterOpts); // Delete config - deleteAndVerifyConfigValue(client, defaultBrokerId, singleton(MESSAGE_MAX_BYTES_CONFIG), true, alterOpts); + deleteAndVerifyConfigValue(client, defaultBrokerId, Set.of(MESSAGE_MAX_BYTES_CONFIG), true, alterOpts); // Listener configs: should work only with listener name alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts); + Map.of("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts); // Per-broker config configured at default cluster-level should fail assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.empty(), - singletonMap("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts)); + Map.of("listener.name.internal.ssl.keystore.location", "/tmp/test.jks"), alterOpts)); deleteAndVerifyConfigValue(client, defaultBrokerId, - singleton("listener.name.internal.ssl.keystore.location"), false, alterOpts); + Set.of("listener.name.internal.ssl.keystore.location"), false, alterOpts); alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap("listener.name.external.ssl.keystore.password", "secret"), alterOpts); + Map.of("listener.name.external.ssl.keystore.password", "secret"), alterOpts); Map configs = new HashMap<>(); configs.put("listener.name.external.ssl.keystore.password", "secret"); @@ -281,12 +277,11 @@ public class ConfigCommandIntegrationTest { @ClusterTest public void testGroupConfigUpdateUsingKraft() throws Exception { List alterOpts = Stream.concat(entityOp(Optional.of(defaultGroupName)).stream(), - Stream.of("--entity-type", "groups", "--alter")) - .collect(Collectors.toList()); + Stream.of("--entity-type", "groups", "--alter")).toList(); verifyGroupConfigUpdate(alterOpts); // Test for the --group alias - verifyGroupConfigUpdate(asList("--group", defaultGroupName, "--alter")); + verifyGroupConfigUpdate(List.of("--group", defaultGroupName, "--alter")); } @ClusterTest @@ -343,7 +338,7 @@ public class ConfigCommandIntegrationTest { deleteAndVerifyGroupConfigValue(client, defaultGroupName, configs, alterOpts); // Unknown config configured should fail - assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, singletonMap("unknown.config", "20000"), alterOpts)); + assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Map.of("unknown.config", "20000"), alterOpts)); } } @@ -351,12 +346,11 @@ public class ConfigCommandIntegrationTest { @ClusterTest(types = {Type.KRAFT}) public void testClientMetricsConfigUpdate() throws Exception { List alterOpts = Stream.concat(entityOp(Optional.of(defaultClientMetricsName)).stream(), - Stream.of("--entity-type", "client-metrics", "--alter")) - .collect(Collectors.toList()); + Stream.of("--entity-type", "client-metrics", "--alter")).toList(); verifyClientMetricsConfigUpdate(alterOpts); // Test for the --client-metrics alias - verifyClientMetricsConfigUpdate(asList("--client-metrics", defaultClientMetricsName, "--alter")); + verifyClientMetricsConfigUpdate(List.of("--client-metrics", defaultClientMetricsName, "--alter")); } private void verifyClientMetricsConfigUpdate(List alterOpts) throws Exception { @@ -373,7 +367,7 @@ public class ConfigCommandIntegrationTest { deleteAndVerifyClientMetricsConfigValue(client, defaultClientMetricsName, configs, alterOpts); // Unknown config configured should fail - assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, singletonMap("unknown.config", "20000"), alterOpts)); + assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Map.of("unknown.config", "20000"), alterOpts)); } } @@ -384,13 +378,13 @@ public class ConfigCommandIntegrationTest { try (Admin client = cluster.admin()) { assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"), alterOpts)); + Map.of(AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false"), alterOpts)); assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false"), alterOpts)); + Map.of(AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false"), alterOpts)); assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap("broker.id", "1"), alterOpts)); + Map.of("broker.id", "1"), alterOpts)); } } @@ -400,11 +394,11 @@ public class ConfigCommandIntegrationTest { try (Admin client = cluster.admin()) { alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap("log.flush.interval.messages", "100"), alterOpts); + Map.of("log.flush.interval.messages", "100"), alterOpts); alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap("log.retention.bytes", "20"), alterOpts); + Map.of("log.retention.bytes", "20"), alterOpts); alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap("log.retention.ms", "2"), alterOpts); + Map.of("log.retention.ms", "2"), alterOpts); } } @@ -415,13 +409,13 @@ public class ConfigCommandIntegrationTest { try (Admin client = cluster.admin()) { alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap(listenerName + "ssl.truststore.type", "PKCS12"), alterOpts); + Map.of(listenerName + "ssl.truststore.type", "PKCS12"), alterOpts); alterAndVerifyConfig(client, Optional.of(defaultBrokerId), - singletonMap(listenerName + "ssl.truststore.location", "/temp/test.jks"), alterOpts); + Map.of(listenerName + "ssl.truststore.location", "/temp/test.jks"), alterOpts); alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(listenerName + "ssl.truststore.password", "password"), alterOpts); + Map.of(listenerName + "ssl.truststore.password", "password"), alterOpts); verifyConfigSecretValue(client, Optional.of(defaultBrokerId), - singleton(listenerName + "ssl.truststore.password")); + Set.of(listenerName + "ssl.truststore.password")); } } @@ -432,13 +426,13 @@ public class ConfigCommandIntegrationTest { try (Admin client = cluster.admin()) { assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"), alterOpts)); + Map.of(SSL_TRUSTSTORE_TYPE_CONFIG, "PKCS12"), alterOpts)); assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(SSL_TRUSTSTORE_LOCATION_CONFIG, "/temp/test.jks"), alterOpts)); + Map.of(SSL_TRUSTSTORE_LOCATION_CONFIG, "/temp/test.jks"), alterOpts)); assertThrows(ExecutionException.class, () -> alterConfigWithAdmin(client, Optional.of(defaultBrokerId), - singletonMap(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password"), alterOpts)); + Map.of(SSL_TRUSTSTORE_PASSWORD_CONFIG, "password"), alterOpts)); } } @@ -469,7 +463,7 @@ public class ConfigCommandIntegrationTest { @ClusterTest public void testUpdateInvalidTopicConfigs() throws ExecutionException, InterruptedException { - List alterOpts = asList("--bootstrap-server", cluster.bootstrapServers(), "--entity-type", "topics", "--alter"); + List alterOpts = List.of("--bootstrap-server", cluster.bootstrapServers(), "--entity-type", "topics", "--alter"); try (Admin client = cluster.admin()) { client.createTopics(List.of(new NewTopic("test-config-topic", 1, (short) 1))).all().get(); assertInstanceOf( @@ -480,7 +474,7 @@ public class ConfigCommandIntegrationTest { client, new ConfigCommand.ConfigCommandOptions( toArray(alterOpts, - asList("--add-config", "invalid=2", "--entity-type", "topics", "--entity-name", "test-config-topic")))) + List.of("--add-config", "invalid=2", "--entity-type", "topics", "--entity-name", "test-config-topic")))) ).getCause() ); } @@ -494,14 +488,14 @@ public class ConfigCommandIntegrationTest { public void testUpdateBrokerConfigNotAffectedByInvalidConfig() { try (Admin client = cluster.admin()) { ConfigCommand.alterConfig(client, new ConfigCommand.ConfigCommandOptions( - toArray(asList("--bootstrap-server", cluster.bootstrapServers(), + toArray(List.of("--bootstrap-server", cluster.bootstrapServers(), "--alter", "--add-config", "log.cleaner.threadzz=2", "--entity-type", "brokers", "--entity-default")))); ConfigCommand.alterConfig(client, new ConfigCommand.ConfigCommandOptions( - toArray(asList("--bootstrap-server", cluster.bootstrapServers(), + toArray(List.of("--bootstrap-server", cluster.bootstrapServers(), "--alter", "--add-config", "log.cleaner.threads=2", "--entity-type", "brokers", @@ -531,15 +525,15 @@ public class ConfigCommandIntegrationTest { .incrementalAlterConfigs(anyMap(), any(AlterConfigsOptions.class)); assertEquals( "The INCREMENTAL_ALTER_CONFIGS API is not supported by the cluster. The API is supported starting from version 2.3.0. You may want to use an older version of this tool to interact with your cluster, or upgrade your brokers to version 2.3.0 or newer to avoid this error.", - assertThrows(UnsupportedVersionException.class, () -> { + assertThrows(UnsupportedVersionException.class, () -> ConfigCommand.alterConfig(spyAdmin, new ConfigCommand.ConfigCommandOptions( - toArray(asList( + toArray(List.of( "--bootstrap-server", cluster.bootstrapServers(), "--alter", "--add-config", "log.cleaner.threads=2", "--entity-type", "brokers", - "--entity-default")))); - }).getMessage() + "--entity-default")))) + ).getMessage() ); Mockito.verify(spyAdmin).incrementalAlterConfigs(anyMap(), any(AlterConfigsOptions.class)); } @@ -585,12 +579,12 @@ public class ConfigCommandIntegrationTest { } private List entityOp(Optional entityId) { - return entityId.map(id -> asList("--entity-name", id)) - .orElse(singletonList("--entity-default")); + return entityId.map(id -> List.of("--entity-name", id)) + .orElse(List.of("--entity-default")); } private List generateDefaultAlterOpts(String bootstrapServers) { - return asList("--bootstrap-server", bootstrapServers, + return List.of("--bootstrap-server", bootstrapServers, "--entity-type", "brokers", "--alter"); } @@ -620,23 +614,23 @@ public class ConfigCommandIntegrationTest { private void alterConfigWithAdmin(Admin client, Optional resourceName, Map config, List alterOpts) { String configStr = transferConfigMapToString(config); - List bootstrapOpts = quorumArgs().collect(Collectors.toList()); + List bootstrapOpts = quorumArgs().toList(); ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts, entityOp(resourceName), alterOpts, - asList("--add-config", configStr))); + List.of("--add-config", configStr))); addOpts.checkArgs(); ConfigCommand.alterConfig(client, addOpts); } private void alterConfigWithAdmin(Admin client, Map config, List alterOpts) { String configStr = transferConfigMapToString(config); - List bootstrapOpts = quorumArgs().collect(Collectors.toList()); + List bootstrapOpts = quorumArgs().toList(); ConfigCommand.ConfigCommandOptions addOpts = new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts, alterOpts, - asList("--add-config", configStr))); + List.of("--add-config", configStr))); addOpts.checkArgs(); ConfigCommand.alterConfig(client, addOpts); } @@ -675,7 +669,7 @@ public class ConfigCommandIntegrationTest { private Stream getConfigEntryStream(Admin client, ConfigResource configResource) throws InterruptedException, ExecutionException { - return client.describeConfigs(singletonList(configResource)) + return client.describeConfigs(List.of(configResource)) .all() .get() .values() @@ -689,8 +683,8 @@ public class ConfigCommandIntegrationTest { boolean hasDefaultValue, List alterOpts) throws Exception { ConfigCommand.ConfigCommandOptions deleteOpts = - new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, asList("--entity-name", brokerId), - asList("--delete-config", String.join(",", config)))); + new ConfigCommand.ConfigCommandOptions(toArray(alterOpts, List.of("--entity-name", brokerId), + List.of("--delete-config", String.join(",", config)))); deleteOpts.checkArgs(); ConfigCommand.alterConfig(client, deleteOpts); verifyPerBrokerConfigValue(client, brokerId, config, hasDefaultValue); @@ -700,11 +694,11 @@ public class ConfigCommandIntegrationTest { String groupName, Map defaultConfigs, List alterOpts) throws Exception { - List bootstrapOpts = quorumArgs().collect(Collectors.toList()); + List bootstrapOpts = quorumArgs().toList(); ConfigCommand.ConfigCommandOptions deleteOpts = new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts, alterOpts, - asList("--delete-config", String.join(",", defaultConfigs.keySet())))); + List.of("--delete-config", String.join(",", defaultConfigs.keySet())))); deleteOpts.checkArgs(); ConfigCommand.alterConfig(client, deleteOpts); verifyGroupConfig(client, groupName, defaultConfigs); @@ -714,11 +708,11 @@ public class ConfigCommandIntegrationTest { String clientMetricsName, Map defaultConfigs, List alterOpts) throws Exception { - List bootstrapOpts = quorumArgs().collect(Collectors.toList()); + List bootstrapOpts = quorumArgs().toList(); ConfigCommand.ConfigCommandOptions deleteOpts = new ConfigCommand.ConfigCommandOptions(toArray(bootstrapOpts, alterOpts, - asList("--delete-config", String.join(",", defaultConfigs.keySet())))); + List.of("--delete-config", String.join(",", defaultConfigs.keySet())))); deleteOpts.checkArgs(); ConfigCommand.alterConfig(client, deleteOpts); verifyClientMetricsConfig(client, clientMetricsName, defaultConfigs); diff --git a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandTest.java index fc0add1022b..0aecba43463 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConfigCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConfigCommandTest.java @@ -53,7 +53,6 @@ import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.io.IOException; import java.util.AbstractMap.SimpleImmutableEntry; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -85,8 +84,8 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class ConfigCommandTest { - private static final List BROKER_BOOTSTRAP = Arrays.asList("--bootstrap-server", "localhost:9092"); - private static final List CONTROLLER_BOOTSTRAP = Arrays.asList("--bootstrap-controller", "localhost:9093"); + private static final List BROKER_BOOTSTRAP = List.of("--bootstrap-server", "localhost:9092"); + private static final List CONTROLLER_BOOTSTRAP = List.of("--bootstrap-controller", "localhost:9093"); @Test public void shouldExitWithNonZeroStatusOnArgError() { @@ -95,7 +94,7 @@ public class ConfigCommandTest { @Test public void shouldExitWithNonZeroStatusAlterUserQuotaWithoutEntityName() { - assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, Arrays.asList( + assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, List.of( "--entity-type", "users", "--alter", "--add-config", "consumer_byte_rate=20000"))); } @@ -110,7 +109,7 @@ public class ConfigCommandTest { @Test public void shouldExitWithNonZeroStatusIfBothBootstrapServerAndBootstrapControllerGiven() { - assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, CONTROLLER_BOOTSTRAP, Arrays.asList( + assertNonZeroStatusExit(toArray(BROKER_BOOTSTRAP, CONTROLLER_BOOTSTRAP, List.of( "--describe", "--broker-defaults"))); } @@ -405,9 +404,9 @@ public class ConfigCommandTest { File file = TestUtils.tempFile(fileContents); - List addConfigFileArgs = Arrays.asList("--add-config-file", file.getPath()); + List addConfigFileArgs = List.of("--add-config-file", file.getPath()); - ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", + ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--entity-name", "1", "--entity-type", "brokers", "--alter"), @@ -423,7 +422,7 @@ public class ConfigCommandTest { } public void testExpectedEntityTypeNames(List expectedTypes, List expectedNames, List connectOpts, String... args) { - ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList(connectOpts.get(0), connectOpts.get(1), "--describe"), Arrays.asList(args))); + ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of(connectOpts.get(0), connectOpts.get(1), "--describe"), List.of(args))); createOpts.checkArgs(); assertEquals(createOpts.entityTypes().toSeq(), seq(expectedTypes)); assertEquals(createOpts.entityNames().toSeq(), seq(expectedNames)); @@ -431,31 +430,31 @@ public class ConfigCommandTest { @Test public void testOptionEntityTypeNames() { - List connectOpts = Arrays.asList("--bootstrap-server", "localhost:9092"); + List connectOpts = List.of("--bootstrap-server", "localhost:9092"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "topics", "--entity-name", "A"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.singletonList("1.2.3.4"), connectOpts, "--entity-name", "1.2.3.4", "--entity-type", "ips"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.CLIENT_METRICS.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "client-metrics", "--entity-name", "A"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.singletonList("A"), connectOpts, "--entity-type", "groups", "--entity-name", "A"); - testExpectedEntityTypeNames(Arrays.asList(ConfigType.USER.value(), ConfigType.CLIENT.value()), Arrays.asList("A", ""), connectOpts, + testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of("A"), connectOpts, "--entity-type", "topics", "--entity-name", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of("1.2.3.4"), connectOpts, "--entity-name", "1.2.3.4", "--entity-type", "ips"); + testExpectedEntityTypeNames(List.of(ConfigType.CLIENT_METRICS.value()), List.of("A"), connectOpts, "--entity-type", "client-metrics", "--entity-name", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of("A"), connectOpts, "--entity-type", "groups", "--entity-name", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.USER.value(), ConfigType.CLIENT.value()), List.of("A", ""), connectOpts, "--entity-type", "users", "--entity-type", "clients", "--entity-name", "A", "--entity-default"); - testExpectedEntityTypeNames(Arrays.asList(ConfigType.USER.value(), ConfigType.CLIENT.value()), Arrays.asList("", "B"), connectOpts, + testExpectedEntityTypeNames(List.of(ConfigType.USER.value(), ConfigType.CLIENT.value()), List.of("", "B"), connectOpts, "--entity-default", "--entity-name", "B", "--entity-type", "users", "--entity-type", "clients"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.singletonList("A"), connectOpts, "--topic", "A"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.singletonList("1.2.3.4"), connectOpts, "--ip", "1.2.3.4"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.singletonList("A"), connectOpts, "--group", "A"); - testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Arrays.asList("B", "A"), connectOpts, "--client", "B", "--user", "A"); - testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Arrays.asList("B", ""), connectOpts, "--client", "B", "--user-defaults"); - testExpectedEntityTypeNames(Arrays.asList(ConfigType.CLIENT.value(), ConfigType.USER.value()), Collections.singletonList("A"), connectOpts, + testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of("A"), connectOpts, "--topic", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of("1.2.3.4"), connectOpts, "--ip", "1.2.3.4"); + testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of("A"), connectOpts, "--group", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("B", "A"), connectOpts, "--client", "B", "--user", "A"); + testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("B", ""), connectOpts, "--client", "B", "--user-defaults"); + testExpectedEntityTypeNames(List.of(ConfigType.CLIENT.value(), ConfigType.USER.value()), List.of("A"), connectOpts, "--entity-type", "clients", "--entity-type", "users", "--entity-name", "A"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.TOPIC.value()), Collections.emptyList(), connectOpts, "--entity-type", "topics"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.IP.value()), Collections.emptyList(), connectOpts, "--entity-type", "ips"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.GROUP.value()), Collections.emptyList(), connectOpts, "--entity-type", "groups"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.CLIENT_METRICS.value()), Collections.emptyList(), connectOpts, "--entity-type", "client-metrics"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.singletonList("0"), connectOpts, "--entity-name", "0", "--entity-type", "brokers"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.singletonList("0"), connectOpts, "--broker", "0"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.USER.value()), Collections.emptyList(), connectOpts, "--entity-type", "users"); - testExpectedEntityTypeNames(Collections.singletonList(ConfigType.BROKER.value()), Collections.emptyList(), connectOpts, "--entity-type", "brokers"); + testExpectedEntityTypeNames(List.of(ConfigType.TOPIC.value()), List.of(), connectOpts, "--entity-type", "topics"); + testExpectedEntityTypeNames(List.of(ConfigType.IP.value()), List.of(), connectOpts, "--entity-type", "ips"); + testExpectedEntityTypeNames(List.of(ConfigType.GROUP.value()), List.of(), connectOpts, "--entity-type", "groups"); + testExpectedEntityTypeNames(List.of(ConfigType.CLIENT_METRICS.value()), List.of(), connectOpts, "--entity-type", "client-metrics"); + testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of("0"), connectOpts, "--entity-name", "0", "--entity-type", "brokers"); + testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of("0"), connectOpts, "--broker", "0"); + testExpectedEntityTypeNames(List.of(ConfigType.USER.value()), List.of(), connectOpts, "--entity-type", "users"); + testExpectedEntityTypeNames(List.of(ConfigType.BROKER.value()), List.of(), connectOpts, "--entity-type", "brokers"); } @Test @@ -501,31 +500,23 @@ public class ConfigCommandTest { } private Entry, Map> argsAndExpectedEntity(Optional entityName, String entityType) { - String command; - switch (entityType) { - case ClientQuotaEntity.USER: - command = "users"; - break; - case ClientQuotaEntity.CLIENT_ID: - command = "clients"; - break; - case ClientQuotaEntity.IP: - command = "ips"; - break; - default: - throw new IllegalArgumentException("Unknown command: " + entityType); - } + String command = switch (entityType) { + case ClientQuotaEntity.USER -> ConfigType.USER.value(); + case ClientQuotaEntity.CLIENT_ID -> ConfigType.CLIENT.value(); + case ClientQuotaEntity.IP -> ConfigType.IP.value(); + default -> throw new IllegalArgumentException("Unknown command: " + entityType); + }; return entityName.map(name -> { if (name.isEmpty()) - return new SimpleImmutableEntry<>(Arrays.asList("--entity-type", command, "--entity-default"), Collections.singletonMap(entityType, (String) null)); - return new SimpleImmutableEntry<>(Arrays.asList("--entity-type", command, "--entity-name", name), Collections.singletonMap(entityType, name)); - }).orElse(new SimpleImmutableEntry<>(Collections.emptyList(), Collections.emptyMap())); + return new SimpleImmutableEntry<>(List.of("--entity-type", command, "--entity-default"), Collections.singletonMap(entityType, (String) null)); + return new SimpleImmutableEntry<>(List.of("--entity-type", command, "--entity-name", name), Map.of(entityType, name)); + }).orElse(new SimpleImmutableEntry<>(List.of(), Map.of())); } private void verifyAlterCommandFails(String expectedErrorMessage, List alterOpts) { Admin mockAdminClient = mock(Admin.class); - ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", + ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--alter"), alterOpts)); IllegalArgumentException e = assertThrows(IllegalArgumentException.class, () -> ConfigCommand.alterConfig(mockAdminClient, opts)); assertTrue(e.getMessage().contains(expectedErrorMessage), "Unexpected exception: " + e); @@ -535,25 +526,25 @@ public class ConfigCommandTest { public void shouldNotAlterNonQuotaIpConfigsUsingBootstrapServer() { // when using --bootstrap-server, it should be illegal to alter anything that is not a connection quota // for ip entities - List ipEntityOpts = Arrays.asList("--entity-type", "ips", "--entity-name", "127.0.0.1"); + List ipEntityOpts = List.of("--entity-type", "ips", "--entity-name", "127.0.0.1"); String invalidProp = "some_config"; - verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--add-config", "connection_creation_rate=10000,some_config=10"))); - verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--add-config", "some_config=10"))); - verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--delete-config", "connection_creation_rate=10000,some_config=10"))); - verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, Arrays.asList("--delete-config", "some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--add-config", "connection_creation_rate=10000,some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--add-config", "some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--delete-config", "connection_creation_rate=10000,some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(ipEntityOpts, List.of("--delete-config", "some_config=10"))); } private void verifyDescribeQuotas(List describeArgs, ClientQuotaFilter expectedFilter) { - ConfigCommand.ConfigCommandOptions describeOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", + ConfigCommand.ConfigCommandOptions describeOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--describe"), describeArgs)); KafkaFutureImpl>> describeFuture = new KafkaFutureImpl<>(); - describeFuture.complete(Collections.emptyMap()); + describeFuture.complete(Map.of()); DescribeClientQuotasResult describeResult = mock(DescribeClientQuotasResult.class); when(describeResult.entities()).thenReturn(describeFuture); AtomicBoolean describedConfigs = new AtomicBoolean(); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) { assertTrue(filter.strict()); @@ -570,24 +561,24 @@ public class ConfigCommandTest { public void testDescribeIpConfigs() { String entityType = ClientQuotaEntity.IP; String knownHost = "1.2.3.4"; - ClientQuotaFilter defaultIpFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofDefaultEntity(entityType))); - ClientQuotaFilter singleIpFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofEntity(entityType, knownHost))); - ClientQuotaFilter allIpsFilter = ClientQuotaFilter.containsOnly(Collections.singletonList(ClientQuotaFilterComponent.ofEntityType(entityType))); - verifyDescribeQuotas(Arrays.asList("--entity-default", "--entity-type", "ips"), defaultIpFilter); - verifyDescribeQuotas(Collections.singletonList("--ip-defaults"), defaultIpFilter); - verifyDescribeQuotas(Arrays.asList("--entity-type", "ips", "--entity-name", knownHost), singleIpFilter); - verifyDescribeQuotas(Arrays.asList("--ip", knownHost), singleIpFilter); - verifyDescribeQuotas(Arrays.asList("--entity-type", "ips"), allIpsFilter); + ClientQuotaFilter defaultIpFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofDefaultEntity(entityType))); + ClientQuotaFilter singleIpFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofEntity(entityType, knownHost))); + ClientQuotaFilter allIpsFilter = ClientQuotaFilter.containsOnly(List.of(ClientQuotaFilterComponent.ofEntityType(entityType))); + verifyDescribeQuotas(List.of("--entity-default", "--entity-type", "ips"), defaultIpFilter); + verifyDescribeQuotas(List.of("--ip-defaults"), defaultIpFilter); + verifyDescribeQuotas(List.of("--entity-type", "ips", "--entity-name", knownHost), singleIpFilter); + verifyDescribeQuotas(List.of("--ip", knownHost), singleIpFilter); + verifyDescribeQuotas(List.of("--entity-type", "ips"), allIpsFilter); } public void verifyAlterQuotas(List alterOpts, ClientQuotaEntity expectedAlterEntity, Map expectedProps, Set expectedAlterOps) { - ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", + ConfigCommand.ConfigCommandOptions createOpts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--alter"), alterOpts)); AtomicBoolean describedConfigs = new AtomicBoolean(); KafkaFutureImpl>> describeFuture = new KafkaFutureImpl<>(); - describeFuture.complete(Collections.singletonMap(expectedAlterEntity, expectedProps)); + describeFuture.complete(Map.of(expectedAlterEntity, expectedProps)); DescribeClientQuotasResult describeResult = mock(DescribeClientQuotasResult.class); when(describeResult.entities()).thenReturn(describeFuture); @@ -606,7 +597,7 @@ public class ConfigCommandTest { when(alterResult.all()).thenReturn(alterFuture); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) { assertTrue(filter.strict()); @@ -638,11 +629,11 @@ public class ConfigCommandTest { Entry, Map> defaultIpArgsAndEntity = argsAndExpectedEntity(Optional.of(""), ClientQuotaEntity.IP); - List deleteArgs = Arrays.asList("--delete-config", "connection_creation_rate"); + List deleteArgs = List.of("--delete-config", "connection_creation_rate"); Set deleteAlterationOps = Set.of(new ClientQuotaAlteration.Op("connection_creation_rate", null)); - Map propsToDelete = Collections.singletonMap("connection_creation_rate", 50.0); + Map propsToDelete = Map.of("connection_creation_rate", 50.0); - List addArgs = Arrays.asList("--add-config", "connection_creation_rate=100"); + List addArgs = List.of("--add-config", "connection_creation_rate=100"); Set addAlterationOps = Set.of(new ClientQuotaAlteration.Op("connection_creation_rate", 100.0)); verifyAlterQuotas( @@ -653,7 +644,7 @@ public class ConfigCommandTest { verifyAlterQuotas( concat(singleIpArgsAndEntity.getKey(), addArgs), new ClientQuotaEntity(singleIpArgsAndEntity.getValue()), - Collections.emptyMap(), + Map.of(), addAlterationOps); verifyAlterQuotas( concat(defaultIpArgsAndEntity.getKey(), deleteArgs), @@ -663,14 +654,14 @@ public class ConfigCommandTest { verifyAlterQuotas( concat(defaultIpArgsAndEntity.getKey(), addArgs), new ClientQuotaEntity(defaultIpArgsAndEntity.getValue()), - Collections.emptyMap(), + Map.of(), addAlterationOps); } private void verifyAlterUserClientQuotas(String user, String client) { - List alterArgs = Arrays.asList("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000", + List alterArgs = List.of("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000", "--delete-config", "request_percentage"); - Map propsToDelete = Collections.singletonMap("request_percentage", 50.0); + Map propsToDelete = Map.of("request_percentage", 50.0); Set alterationOps = Set.of( new ClientQuotaAlteration.Op("consumer_byte_rate", 20000d), @@ -700,10 +691,10 @@ public class ConfigCommandTest { verifyAlterUserClientQuotas(null, ""); } - private final List userEntityOpts = Arrays.asList("--entity-type", "users", "--entity-name", "admin"); - private final List clientEntityOpts = Arrays.asList("--entity-type", "clients", "--entity-name", "admin"); - private final List addScramOpts = Arrays.asList("--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]"); - private final List deleteScramOpts = Arrays.asList("--delete-config", "SCRAM-SHA-256"); + private final List userEntityOpts = List.of("--entity-type", "users", "--entity-name", "admin"); + private final List clientEntityOpts = List.of("--entity-type", "clients", "--entity-name", "admin"); + private final List addScramOpts = List.of("--add-config", "SCRAM-SHA-256=[iterations=8192,password=foo-secret]"); + private final List deleteScramOpts = List.of("--delete-config", "SCRAM-SHA-256"); @Test public void shouldNotAlterNonQuotaNonScramUserOrClientConfigUsingBootstrapServer() { @@ -711,13 +702,13 @@ public class ConfigCommandTest { // for both user and client entities String invalidProp = "some_config"; verifyAlterCommandFails(invalidProp, concat(userEntityOpts, - Arrays.asList("-add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10"))); + List.of("-add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10"))); verifyAlterCommandFails(invalidProp, concat(userEntityOpts, - Arrays.asList("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10"))); - verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, Arrays.asList("--add-config", "some_config=10"))); - verifyAlterCommandFails(invalidProp, concat(userEntityOpts, Arrays.asList("--delete-config", "consumer_byte_rate,some_config"))); - verifyAlterCommandFails(invalidProp, concat(userEntityOpts, Arrays.asList("--delete-config", "SCRAM-SHA-256,some_config"))); - verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, Arrays.asList("--delete-config", "some_config"))); + List.of("--add-config", "consumer_byte_rate=20000,producer_byte_rate=10000,some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, List.of("--add-config", "some_config=10"))); + verifyAlterCommandFails(invalidProp, concat(userEntityOpts, List.of("--delete-config", "consumer_byte_rate,some_config"))); + verifyAlterCommandFails(invalidProp, concat(userEntityOpts, List.of("--delete-config", "SCRAM-SHA-256,some_config"))); + verifyAlterCommandFails(invalidProp, concat(clientEntityOpts, List.of("--delete-config", "some_config"))); } @Test @@ -731,16 +722,16 @@ public class ConfigCommandTest { public void shouldNotCreateUserScramCredentialConfigWithUnderMinimumIterationsUsingBootstrapServer() { // when using --bootstrap-server, it should be illegal to create a SCRAM credential for a user // with an iterations value less than the minimum - verifyAlterCommandFails("SCRAM-SHA-256", concat(userEntityOpts, Arrays.asList("--add-config", "SCRAM-SHA-256=[iterations=100,password=foo-secret]"))); + verifyAlterCommandFails("SCRAM-SHA-256", concat(userEntityOpts, List.of("--add-config", "SCRAM-SHA-256=[iterations=100,password=foo-secret]"))); } @Test public void shouldNotAlterUserScramCredentialAndClientQuotaConfigsSimultaneouslyUsingBootstrapServer() { // when using --bootstrap-server, it should be illegal to alter both SCRAM credentials and quotas for user entities String expectedErrorMessage = "SCRAM-SHA-256"; - List secondUserEntityOpts = Arrays.asList("--entity-type", "users", "--entity-name", "admin1"); - List addQuotaOpts = Arrays.asList("--add-config", "consumer_byte_rate=20000"); - List deleteQuotaOpts = Arrays.asList("--delete-config", "consumer_byte_rate"); + List secondUserEntityOpts = List.of("--entity-type", "users", "--entity-name", "admin1"); + List addQuotaOpts = List.of("--add-config", "consumer_byte_rate=20000"); + List deleteQuotaOpts = List.of("--delete-config", "consumer_byte_rate"); verifyAlterCommandFails(expectedErrorMessage, concat(userEntityOpts, addScramOpts, userEntityOpts, deleteQuotaOpts)); verifyAlterCommandFails(expectedErrorMessage, concat(userEntityOpts, addScramOpts, secondUserEntityOpts, deleteQuotaOpts)); @@ -758,11 +749,11 @@ public class ConfigCommandTest { // User SCRAM credentials should not be described when specifying // --describe --entity-type users --entity-default (or --user-defaults) with --bootstrap-server KafkaFutureImpl>> describeFuture = new KafkaFutureImpl<>(); - describeFuture.complete(Collections.singletonMap(new ClientQuotaEntity(Collections.singletonMap("", "")), Collections.singletonMap("request_percentage", 50.0))); + describeFuture.complete(Map.of(new ClientQuotaEntity(Map.of("", "")), Map.of("request_percentage", 50.0))); DescribeClientQuotasResult describeClientQuotasResult = mock(DescribeClientQuotasResult.class); when(describeClientQuotasResult.entities()).thenReturn(describeFuture); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public DescribeClientQuotasResult describeClientQuotas(ClientQuotaFilter filter, DescribeClientQuotasOptions options) { return describeClientQuotasResult; @@ -773,15 +764,15 @@ public class ConfigCommandTest { throw new IllegalStateException("Incorrectly described SCRAM credentials when specifying --entity-default with --bootstrap-server"); } }; - ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(Arrays.asList("--bootstrap-server", "localhost:9092", "--describe"), requestOpts)); + ConfigCommand.ConfigCommandOptions opts = new ConfigCommand.ConfigCommandOptions(toArray(List.of("--bootstrap-server", "localhost:9092", "--describe"), requestOpts)); ConfigCommand.describeConfig(mockAdminClient, opts); // fails if describeUserScramCredentials() is invoked } @Test public void shouldNotDescribeUserScramCredentialsWithEntityDefaultUsingBootstrapServer() { String expectedMsg = "The use of --entity-default or --user-defaults is not allowed with User SCRAM Credentials using --bootstrap-server."; - List defaultUserOpt = Collections.singletonList("--user-defaults"); - List verboseDefaultUserOpts = Arrays.asList("--entity-type", "users", "--entity-default"); + List defaultUserOpt = List.of("--user-defaults"); + List verboseDefaultUserOpts = List.of("--entity-type", "users", "--entity-default"); verifyAlterCommandFails(expectedMsg, concat(verboseDefaultUserOpts, addScramOpts)); verifyAlterCommandFails(expectedMsg, concat(verboseDefaultUserOpts, deleteScramOpts)); verifyUserScramCredentialsNotDescribed(verboseDefaultUserOpts); @@ -813,9 +804,9 @@ public class ConfigCommandTest { AtomicBoolean alteredConfigs = new AtomicBoolean(); ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName); - List configEntries = Arrays.asList(newConfigEntry("min.insync.replicas", "1"), newConfigEntry("unclean.leader.election.enable", "1")); + List configEntries = List.of(newConfigEntry("min.insync.replicas", "1"), newConfigEntry("unclean.leader.election.enable", "1")); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(configEntries))); + future.complete(Map.of(resource, new Config(configEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); @@ -825,14 +816,14 @@ public class ConfigCommandTest { when(alterResult.all()).thenReturn(alterFuture); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily"); assertEquals(1, resources.size()); ConfigResource res = resources.iterator().next(); - assertEquals(res.type(), ConfigResource.Type.TOPIC); - assertEquals(res.name(), resourceName); + assertEquals(ConfigResource.Type.TOPIC, res.type()); + assertEquals(resourceName, res.name()); return describeResult; } @@ -869,7 +860,7 @@ public class ConfigCommandTest { } public ConfigEntry newConfigEntry(String name, String value) { - return ConfigTest.newConfigEntry(name, value, ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, Collections.emptyList()); + return ConfigTest.newConfigEntry(name, value, ConfigEntry.ConfigSource.DYNAMIC_TOPIC_CONFIG, false, false, List.of()); } @Test @@ -883,16 +874,16 @@ public class ConfigCommandTest { ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(Collections.emptyList()))); + future.complete(Map.of(resource, new Config(List.of()))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertTrue(options.includeSynonyms(), "Synonyms not requested"); - assertEquals(Collections.singleton(resource), new HashSet<>(resources)); + assertEquals(Set.of(resource), new HashSet<>(resources)); return describeResult; } }; @@ -903,7 +894,7 @@ public class ConfigCommandTest { @Test public void shouldAddBrokerLoggerConfig() { Node node = new Node(1, "localhost", 9092); - verifyAlterBrokerLoggerConfig(node, "1", "1", Arrays.asList( + verifyAlterBrokerLoggerConfig(node, "1", "1", List.of( new ConfigEntry("kafka.log.LogCleaner", "INFO"), new ConfigEntry("kafka.server.ReplicaManager", "INFO"), new ConfigEntry("kafka.server.KafkaApi", "INFO") @@ -970,7 +961,7 @@ public class ConfigCommandTest { Node node = new Node(1, "localhost", 9092); // verifyAlterBrokerLoggerConfig tries to alter kafka.log.LogCleaner, kafka.server.ReplicaManager and kafka.server.KafkaApi // yet, we make it so DescribeConfigs returns only one logger, implying that kafka.server.ReplicaManager and kafka.log.LogCleaner are invalid - assertThrows(InvalidConfigurationException.class, () -> verifyAlterBrokerLoggerConfig(node, "1", "1", Collections.singletonList( + assertThrows(InvalidConfigurationException.class, () -> verifyAlterBrokerLoggerConfig(node, "1", "1", List.of( new ConfigEntry("kafka.server.KafkaApi", "INFO") ))); } @@ -978,17 +969,17 @@ public class ConfigCommandTest { @Test public void shouldAddDefaultBrokerDynamicConfig() { Node node = new Node(1, "localhost", 9092); - verifyAlterBrokerConfig(node, "", Collections.singletonList("--entity-default")); + verifyAlterBrokerConfig(node, "", List.of("--entity-default")); } @Test public void shouldAddBrokerDynamicConfig() { Node node = new Node(1, "localhost", 9092); - verifyAlterBrokerConfig(node, "1", Arrays.asList("--entity-name", "1")); + verifyAlterBrokerConfig(node, "1", List.of("--entity-name", "1")); } public void verifyAlterBrokerConfig(Node node, String resourceName, List resourceOpts) { - String[] optsList = toArray(Arrays.asList("--bootstrap-server", "localhost:9092", + String[] optsList = toArray(List.of("--bootstrap-server", "localhost:9092", "--entity-type", "brokers", "--alter", "--add-config", "message.max.bytes=10,leader.replication.throttled.rate=10"), resourceOpts); @@ -997,9 +988,9 @@ public class ConfigCommandTest { brokerConfigs.put("num.io.threads", "5"); ConfigResource resource = new ConfigResource(ConfigResource.Type.BROKER, resourceName); - List configEntries = Collections.singletonList(new ConfigEntry("num.io.threads", "5")); + List configEntries = List.of(new ConfigEntry("num.io.threads", "5")); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(configEntries))); + future.complete(Map.of(resource, new Config(configEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); @@ -1008,7 +999,7 @@ public class ConfigCommandTest { AlterConfigsResult alterResult = mock(AlterConfigsResult.class); when(alterResult.all()).thenReturn(alterFuture); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily"); @@ -1049,7 +1040,7 @@ public class ConfigCommandTest { ConfigResource resourceCustom = new ConfigResource(ConfigResource.Type.BROKER, "1"); ConfigResource resourceDefault = new ConfigResource(ConfigResource.Type.BROKER, brokerDefaultEntityName); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - Config emptyConfig = new Config(Collections.emptyList()); + Config emptyConfig = new Config(List.of()); Map resultMap = new HashMap<>(); resultMap.put(resourceCustom, emptyConfig); resultMap.put(resourceDefault, emptyConfig); @@ -1059,7 +1050,7 @@ public class ConfigCommandTest { when(describeResult.all()).thenReturn(future); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertTrue(options.includeSynonyms(), "Synonyms not requested"); @@ -1087,7 +1078,7 @@ public class ConfigCommandTest { ConfigResource resource = new ConfigResource(ConfigResource.Type.BROKER_LOGGER, resourceName); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(describeConfigEntries))); + future.complete(Map.of(resource, new Config(describeConfigEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); @@ -1096,7 +1087,7 @@ public class ConfigCommandTest { AlterConfigsResult alterResult = mock(AlterConfigsResult.class); when(alterResult.all()).thenReturn(alterFuture); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertEquals(1, resources.size()); @@ -1115,7 +1106,7 @@ public class ConfigCommandTest { assertEquals(ConfigResource.Type.BROKER_LOGGER, res.type()); assertEquals(3, alterConfigOps.size()); - List expectedConfigOps = Arrays.asList( + List expectedConfigOps = List.of( new AlterConfigOp(new ConfigEntry("kafka.server.ReplicaManager", ""), AlterConfigOp.OpType.DELETE), new AlterConfigOp(new ConfigEntry("kafka.server.KafkaApi", ""), AlterConfigOp.OpType.DELETE), new AlterConfigOp(new ConfigEntry("kafka.log.LogCleaner", "DEBUG"), AlterConfigOp.OpType.SET) @@ -1174,14 +1165,14 @@ public class ConfigCommandTest { "--delete-config", "missing_config1, missing_config2")); ConfigResource resource = new ConfigResource(ConfigResource.Type.TOPIC, resourceName); - List configEntries = Collections.emptyList(); + List configEntries = List.of(); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(configEntries))); + future.complete(Map.of(resource, new Config(configEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertEquals(1, resources.size()); @@ -1199,15 +1190,15 @@ public class ConfigCommandTest { @Test public void shouldAlterClientMetricsConfig() { Node node = new Node(1, "localhost", 9092); - verifyAlterClientMetricsConfig(node, "1", Arrays.asList("--entity-type", "client-metrics", "--entity-name", "1")); + verifyAlterClientMetricsConfig(node, "1", List.of("--entity-type", "client-metrics", "--entity-name", "1")); // Test for the --client-metrics alias node = new Node(1, "localhost", 9092); - verifyAlterClientMetricsConfig(node, "1", Arrays.asList("--client-metrics", "1")); + verifyAlterClientMetricsConfig(node, "1", List.of("--client-metrics", "1")); } private void verifyAlterClientMetricsConfig(Node node, String resourceName, List resourceOpts) { - List optsList = concat(Arrays.asList("--bootstrap-server", "localhost:9092", + List optsList = concat(List.of("--bootstrap-server", "localhost:9092", "--alter", "--delete-config", "interval.ms", "--add-config", "metrics=org.apache.kafka.consumer.," + @@ -1215,11 +1206,11 @@ public class ConfigCommandTest { ConfigCommand.ConfigCommandOptions alterOpts = new ConfigCommand.ConfigCommandOptions(toArray(optsList)); ConfigResource resource = new ConfigResource(ConfigResource.Type.CLIENT_METRICS, resourceName); - List configEntries = Collections.singletonList(new ConfigEntry("interval.ms", "1000", - ConfigEntry.ConfigSource.DYNAMIC_CLIENT_METRICS_CONFIG, false, false, Collections.emptyList(), + List configEntries = List.of(new ConfigEntry("interval.ms", "1000", + ConfigEntry.ConfigSource.DYNAMIC_CLIENT_METRICS_CONFIG, false, false, List.of(), ConfigEntry.ConfigType.UNKNOWN, null)); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(configEntries))); + future.complete(Map.of(resource, new Config(configEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); @@ -1228,7 +1219,7 @@ public class ConfigCommandTest { AlterConfigsResult alterResult = mock(AlterConfigsResult.class); when(alterResult.all()).thenReturn(alterFuture); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily"); @@ -1248,7 +1239,7 @@ public class ConfigCommandTest { assertEquals(ConfigResource.Type.CLIENT_METRICS, res.type()); assertEquals(3, alterConfigOps.size()); - List expectedConfigOps = Arrays.asList( + List expectedConfigOps = List.of( new AlterConfigOp(new ConfigEntry("interval.ms", ""), AlterConfigOp.OpType.DELETE), new AlterConfigOp(new ConfigEntry("match", "client_software_name=kafka.python,client_software_version=1\\.2\\..*"), AlterConfigOp.OpType.SET), new AlterConfigOp(new ConfigEntry("metrics", "org.apache.kafka.consumer."), AlterConfigOp.OpType.SET) @@ -1279,7 +1270,7 @@ public class ConfigCommandTest { when(describeResult.all()).thenReturn(future); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertTrue(options.includeSynonyms()); @@ -1287,12 +1278,12 @@ public class ConfigCommandTest { ConfigResource resource = resources.iterator().next(); assertEquals(ConfigResource.Type.CLIENT_METRICS, resource.type()); assertEquals(resourceCustom.name(), resource.name()); - future.complete(Collections.singletonMap(resourceCustom, new Config(Collections.singletonList(configEntry)))); + future.complete(Map.of(resourceCustom, new Config(List.of(configEntry)))); return describeResult; } }; - mockAdminClient.incrementalAlterConfigs(Collections.singletonMap(resourceCustom, - Collections.singletonList(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions()); + mockAdminClient.incrementalAlterConfigs(Map.of(resourceCustom, + List.of(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions()); ConfigCommand.describeConfig(mockAdminClient, describeOpts); verify(describeResult).all(); } @@ -1311,25 +1302,25 @@ public class ConfigCommandTest { @Test public void shouldAlterGroupConfig() { Node node = new Node(1, "localhost", 9092); - verifyAlterGroupConfig(node, "group", Arrays.asList("--entity-type", "groups", "--entity-name", "group")); + verifyAlterGroupConfig(node, "group", List.of("--entity-type", "groups", "--entity-name", "group")); // Test for the --group alias - verifyAlterGroupConfig(node, "groupUsingAlias", Arrays.asList("--group", "groupUsingAlias")); + verifyAlterGroupConfig(node, "groupUsingAlias", List.of("--group", "groupUsingAlias")); } private void verifyAlterGroupConfig(Node node, String resourceName, List resourceOpts) { - List optsList = concat(Arrays.asList("--bootstrap-server", "localhost:9092", + List optsList = concat(List.of("--bootstrap-server", "localhost:9092", "--alter", "--delete-config", "consumer.session.timeout.ms", "--add-config", "consumer.heartbeat.interval.ms=6000"), resourceOpts); ConfigCommand.ConfigCommandOptions alterOpts = new ConfigCommand.ConfigCommandOptions(toArray(optsList)); ConfigResource resource = new ConfigResource(ConfigResource.Type.GROUP, resourceName); - List configEntries = Collections.singletonList(new ConfigEntry("consumer.session.timeout.ms", "45000", - ConfigEntry.ConfigSource.DYNAMIC_GROUP_CONFIG, false, false, Collections.emptyList(), + List configEntries = List.of(new ConfigEntry("consumer.session.timeout.ms", "45000", + ConfigEntry.ConfigSource.DYNAMIC_GROUP_CONFIG, false, false, List.of(), ConfigEntry.ConfigType.UNKNOWN, null)); KafkaFutureImpl> future = new KafkaFutureImpl<>(); - future.complete(Collections.singletonMap(resource, new Config(configEntries))); + future.complete(Map.of(resource, new Config(configEntries))); DescribeConfigsResult describeResult = mock(DescribeConfigsResult.class); when(describeResult.all()).thenReturn(future); @@ -1338,7 +1329,7 @@ public class ConfigCommandTest { AlterConfigsResult alterResult = mock(AlterConfigsResult.class); when(alterResult.all()).thenReturn(alterFuture); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertFalse(options.includeSynonyms(), "Config synonyms requested unnecessarily"); @@ -1358,7 +1349,7 @@ public class ConfigCommandTest { assertEquals(ConfigResource.Type.GROUP, res.type()); assertEquals(2, alterConfigOps.size()); - List expectedConfigOps = Arrays.asList( + List expectedConfigOps = List.of( new AlterConfigOp(new ConfigEntry("consumer.session.timeout.ms", ""), AlterConfigOp.OpType.DELETE), new AlterConfigOp(new ConfigEntry("consumer.heartbeat.interval.ms", "6000"), AlterConfigOp.OpType.SET) ); @@ -1392,7 +1383,7 @@ public class ConfigCommandTest { when(describeResult.all()).thenReturn(future); Node node = new Node(1, "localhost", 9092); - MockAdminClient mockAdminClient = new MockAdminClient(Collections.singletonList(node), node) { + MockAdminClient mockAdminClient = new MockAdminClient(List.of(node), node) { @Override public synchronized DescribeConfigsResult describeConfigs(Collection resources, DescribeConfigsOptions options) { assertTrue(options.includeSynonyms()); @@ -1400,12 +1391,12 @@ public class ConfigCommandTest { ConfigResource resource = resources.iterator().next(); assertEquals(ConfigResource.Type.GROUP, resource.type()); assertEquals(resourceCustom.name(), resource.name()); - future.complete(Collections.singletonMap(resourceCustom, new Config(Collections.singletonList(configEntry)))); + future.complete(Map.of(resourceCustom, new Config(List.of(configEntry)))); return describeResult; } }; - mockAdminClient.incrementalAlterConfigs(Collections.singletonMap(resourceCustom, - Collections.singletonList(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions()); + mockAdminClient.incrementalAlterConfigs(Map.of(resourceCustom, + List.of(new AlterConfigOp(configEntry, AlterConfigOp.OpType.SET))), new AlterConfigsOptions()); ConfigCommand.describeConfig(mockAdminClient, describeOpts); verify(describeResult).all(); } @@ -1432,7 +1423,7 @@ public class ConfigCommandTest { @SafeVarargs public static List concat(List... lists) { - return Stream.of(lists).flatMap(List::stream).collect(Collectors.toList()); + return Stream.of(lists).flatMap(List::stream).toList(); } @SafeVarargs @@ -1448,7 +1439,7 @@ public class ConfigCommandTest { static class DummyAdminClient extends MockAdminClient { public DummyAdminClient(Node node) { - super(Collections.singletonList(node), node); + super(List.of(node), node); } @Override diff --git a/tools/src/test/java/org/apache/kafka/tools/ConnectPluginPathTest.java b/tools/src/test/java/org/apache/kafka/tools/ConnectPluginPathTest.java index f6b70cd9878..2b8666d57bc 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConnectPluginPathTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConnectPluginPathTest.java @@ -568,7 +568,7 @@ public class ConnectPluginPathTest { } catch (IOException e) { throw new UncheckedIOException(e); } - return new WorkerConfig(path, Arrays.asList(pluginPathElements)); + return new WorkerConfig(path, List.of(pluginPathElements)); } private static class CommandResult { diff --git a/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java b/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java index 3804224ef17..6752aef29c7 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ConsoleProducerTest.java @@ -35,9 +35,9 @@ import java.io.OutputStream; import java.nio.file.Files; import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Map; -import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -93,7 +93,7 @@ public class ConsoleProducerTest { ConsoleProducerOptions opts = new ConsoleProducerOptions(BOOTSTRAP_SERVER_VALID_ARGS); ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); - assertEquals(asList("localhost:1003", "localhost:1004"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + assertEquals(List.of("localhost:1003", "localhost:1004"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); } @Test @@ -147,7 +147,7 @@ public class ConsoleProducerTest { ConsoleProducerOptions opts = new ConsoleProducerOptions(BOOTSTRAP_SERVER_OVERRIDE); ProducerConfig producerConfig = new ProducerConfig(opts.producerProps()); - assertEquals(Collections.singletonList("localhost:1002"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); + assertEquals(List.of("localhost:1002"), producerConfig.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/DelegationTokenCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DelegationTokenCommandTest.java index be0b744a7a4..2253d488267 100644 --- a/tools/src/test/java/org/apache/kafka/tools/DelegationTokenCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/DelegationTokenCommandTest.java @@ -24,8 +24,6 @@ import org.apache.kafka.common.utils.Exit; import org.junit.jupiter.api.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutionException; @@ -55,12 +53,12 @@ public class DelegationTokenCommandTest { tokens = DelegationTokenCommand.describeToken(adminClient, getDescribeOpts("")); assertEquals(2, tokens.size()); - assertEquals(Arrays.asList(token1, token2), tokens); + assertEquals(List.of(token1, token2), tokens); //get tokens for renewer2 tokens = DelegationTokenCommand.describeToken(adminClient, getDescribeOpts(renewer2)); assertEquals(1, tokens.size()); - assertEquals(Collections.singletonList(token2), tokens); + assertEquals(List.of(token2), tokens); //test renewing tokens Long expiryTimestamp = DelegationTokenCommand.renewToken(adminClient, getRenewOpts(token1.hmacAsBase64String())); diff --git a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java index f5bab944394..a79f8ef3df3 100644 --- a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java @@ -33,12 +33,11 @@ import org.junit.jupiter.api.Test; import java.io.IOException; import java.nio.file.NoSuchFileException; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -62,7 +61,7 @@ public class DeleteRecordsCommandTest { "Offset json file contains duplicate topic partitions: t-0" ); - admin.createTopics(Collections.singleton(new NewTopic("t", 1, (short) 1))).all().get(); + admin.createTopics(Set.of(new NewTopic("t", 1, (short) 1))).all().get(); Properties props = new Properties(); @@ -159,8 +158,8 @@ public class DeleteRecordsCommandTest { ); assertEquals(2, res.size()); - assertEquals(Arrays.asList(0L, 2L, 0L), res.get(new TopicPartition("t", 0))); - assertEquals(Collections.singletonList(1L), res.get(new TopicPartition("t", 1))); + assertEquals(List.of(0L, 2L, 0L), res.get(new TopicPartition("t", 0))); + assertEquals(List.of(1L), res.get(new TopicPartition("t", 1))); } /** diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index 176a77b99db..19cd8bf2a37 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -28,15 +28,11 @@ import net.sourceforge.argparse4j.inf.Namespace; import org.junit.jupiter.api.Test; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; import static java.lang.String.format; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; import static org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType.SAFE_DOWNGRADE; import static org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType.UNSAFE_DOWNGRADE; import static org.junit.jupiter.api.Assertions.assertArrayEquals; @@ -46,7 +42,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class FeatureCommandTest { - private final List testingFeatures = Arrays.stream(Feature.FEATURES).collect(Collectors.toList()); + private final List testingFeatures = Arrays.stream(Feature.FEATURES).toList(); @ClusterTest(types = {Type.KRAFT}, metadataVersion = MetadataVersion.IBP_3_3_IV3) public void testDescribeWithKRaft(ClusterInstance cluster) { @@ -54,7 +50,7 @@ public class FeatureCommandTest { assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe")) ); - List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + List features = Arrays.stream(commandOutput.split("\n")).sorted().toList(); // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: eligible.leader.replicas.version\tSupportedMinVersion: 0\t" + @@ -80,7 +76,7 @@ public class FeatureCommandTest { assertEquals(0, FeatureCommand.mainNoExit("--bootstrap-controller", cluster.bootstrapControllers(), "describe")) ); - List features = Arrays.stream(commandOutput.split("\n")).sorted().collect(Collectors.toList()); + List features = Arrays.stream(commandOutput.split("\n")).sorted().toList(); // Change expected message to reflect latest MetadataVersion (SupportedMaxVersion increases when adding a new version) assertEquals("Feature: eligible.leader.replicas.version\tSupportedMinVersion: 0\t" + @@ -223,10 +219,10 @@ public class FeatureCommandTest { @Test public void testDowngradeType() { assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType( - new Namespace(singletonMap("unsafe", Boolean.FALSE)))); + new Namespace(Map.of("unsafe", Boolean.FALSE)))); assertEquals(UNSAFE_DOWNGRADE, FeatureCommand.downgradeType( - new Namespace(singletonMap("unsafe", Boolean.TRUE)))); - assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(new Namespace(emptyMap()))); + new Namespace(Map.of("unsafe", Boolean.TRUE)))); + assertEquals(SAFE_DOWNGRADE, FeatureCommand.downgradeType(new Namespace(Map.of()))); } @Test @@ -275,7 +271,7 @@ public class FeatureCommandTest { public void testHandleUpgradeToUnsupportedMetadataVersion() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.3-IV1"); - namespace.put("feature", Collections.singletonList("foo.bar=6")); + namespace.put("feature", List.of("foo.bar=6")); namespace.put("dry_run", false); Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("Unknown metadata.version 3.3-IV1")); @@ -285,7 +281,7 @@ public class FeatureCommandTest { public void testHandleUpgradeToLowerVersion() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.3-IV3"); - namespace.put("feature", Collections.singletonList("foo.bar=6")); + namespace.put("feature", List.of("foo.bar=6")); namespace.put("dry_run", false); String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); @@ -300,7 +296,7 @@ public class FeatureCommandTest { public void testHandleUpgradeToLowerVersionDryRun() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.3-IV3"); - namespace.put("feature", Collections.singletonList("foo.bar=6")); + namespace.put("feature", List.of("foo.bar=6")); namespace.put("dry_run", true); String upgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); @@ -315,7 +311,7 @@ public class FeatureCommandTest { public void testHandleDowngrade() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.7-IV0"); - namespace.put("feature", Collections.singletonList("foo.bar=1")); + namespace.put("feature", List.of("foo.bar=1")); namespace.put("dry_run", false); String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDowngrade(new Namespace(namespace), buildAdminClient())); @@ -330,7 +326,7 @@ public class FeatureCommandTest { public void testHandleDowngradeDryRun() { Map namespace = new HashMap<>(); namespace.put("metadata", "3.7-IV0"); - namespace.put("feature", Collections.singletonList("foo.bar=1")); + namespace.put("feature", List.of("foo.bar=1")); namespace.put("dry_run", true); String downgradeOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDowngrade(new Namespace(namespace), buildAdminClient())); @@ -344,7 +340,7 @@ public class FeatureCommandTest { @Test public void testHandleDisable() { Map namespace = new HashMap<>(); - namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux")); + namespace.put("feature", List.of("foo.bar", "metadata.version", "quux")); namespace.put("dry_run", false); String disableOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient())); @@ -358,7 +354,7 @@ public class FeatureCommandTest { @Test public void testHandleDisableDryRun() { Map namespace = new HashMap<>(); - namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux")); + namespace.put("feature", List.of("foo.bar", "metadata.version", "quux")); namespace.put("dry_run", true); String disableOutput = ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleDisable(new Namespace(namespace), buildAdminClient())); @@ -383,7 +379,7 @@ public class FeatureCommandTest { public void testIncompatibleUpgradeFlags() { Map namespace = new HashMap<>(); namespace.put("release_version", "3.3-IV3"); - namespace.put("feature", Arrays.asList("foo.bar", "metadata.version", "quux")); + namespace.put("feature", List.of("foo.bar", "metadata.version", "quux")); ToolsTestUtils.captureStandardOut(() -> { Throwable t = assertThrows(TerseException.class, () -> FeatureCommand.handleUpgrade(new Namespace(namespace), buildAdminClient())); assertTrue(t.getMessage().contains("Can not specify `release-version` with other feature flags.")); @@ -474,7 +470,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForFeatureWithDependencies() { Map namespace = new HashMap<>(); - namespace.put("feature", Collections.singletonList("test.feature.version=2")); + namespace.put("feature", List.of("test.feature.version=2")); String output = ToolsTestUtils.captureStandardOut(() -> { try { @@ -496,7 +492,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForFeatureWithNoDependencies() { Map namespace = new HashMap<>(); - namespace.put("feature", Collections.singletonList("metadata.version=17")); + namespace.put("feature", List.of("metadata.version=17")); String output = ToolsTestUtils.captureStandardOut(() -> { try { @@ -512,7 +508,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForUnknownFeature() { Map namespace = new HashMap<>(); - namespace.put("feature", Collections.singletonList("unknown.feature=1")); + namespace.put("feature", List.of("unknown.feature=1")); Exception exception = assertThrows( TerseException.class, @@ -525,7 +521,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForFeatureWithUnknownFeatureVersion() { Map namespace = new HashMap<>(); - namespace.put("feature", Collections.singletonList("transaction.version=1000")); + namespace.put("feature", List.of("transaction.version=1000")); Exception exception = assertThrows( IllegalArgumentException.class, @@ -538,7 +534,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForInvalidVersionFormat() { Map namespace = new HashMap<>(); - namespace.put("feature", Collections.singletonList("metadata.version=invalid")); + namespace.put("feature", List.of("metadata.version=invalid")); RuntimeException exception = assertThrows( RuntimeException.class, @@ -554,7 +550,7 @@ public class FeatureCommandTest { @Test public void testHandleFeatureDependenciesForMultipleFeatures() { Map namespace = new HashMap<>(); - namespace.put("feature", Arrays.asList( + namespace.put("feature", List.of( "transaction.version=2", "group.version=1", "test.feature.version=2" diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index 5ee3b385640..9986daa7f3b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -46,7 +46,6 @@ import org.apache.kafka.test.TestUtils; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -85,7 +84,7 @@ public class GetOffsetShellTest { } private void setUp() { - setupTopics(this::getTopicName, Collections.emptyMap()); + setupTopics(this::getTopicName, Map.of()); sendProducerRecords(this::getTopicName); } @@ -139,7 +138,7 @@ public class GetOffsetShellTest { serverProperties.put(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG, "100"); serverProperties.put(RemoteLogManagerConfig.REMOTE_LOG_METADATA_MANAGER_LISTENER_NAME_PROP, "EXTERNAL"); - return Collections.singletonList( + return List.of( ClusterConfig.defaultBuilder() .setTypes(Stream.of(KRAFT, CO_KRAFT).collect(Collectors.toSet())) .setServerProperties(serverProperties) @@ -238,7 +237,7 @@ public class GetOffsetShellTest { setUp(); List offsets = executeAndParse("--partitions", "0,1"); - assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).collect(Collectors.toList()), offsets); + assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).toList(), offsets); } @ClusterTest @@ -247,7 +246,7 @@ public class GetOffsetShellTest { List offsets = executeAndParse("--topic", "topic.*", "--partitions", "0,1"); - assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).collect(Collectors.toList()), offsets); + assertEquals(expectedTestTopicOffsets().stream().filter(r -> r.partition <= 1).toList(), offsets); } @ClusterTest @@ -257,7 +256,7 @@ public class GetOffsetShellTest { createConsumerAndPoll(); List offsets = executeAndParse("--topic-partitions", "topic1:0,topic2:1,topic(3|4):2,__.*:3"); - List expected = Arrays.asList( + List expected = List.of( new Row("__consumer_offsets", 3, 0L), new Row("topic1", 0, 1L), new Row("topic2", 1, 2L), @@ -274,7 +273,7 @@ public class GetOffsetShellTest { for (String time : new String[] {"-1", "latest"}) { List offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time); - List expected = Arrays.asList( + List expected = List.of( new Row("topic1", 0, 1L), new Row("topic2", 0, 2L), new Row("topic3", 0, 3L), @@ -291,7 +290,7 @@ public class GetOffsetShellTest { for (String time : new String[] {"-2", "earliest"}) { List offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time); - List expected = Arrays.asList( + List expected = List.of( new Row("topic1", 0, 0L), new Row("topic2", 0, 0L), new Row("topic3", 0, 0L), @@ -324,7 +323,7 @@ public class GetOffsetShellTest { // test topics disable remote log storage // as remote log disabled, broker return the same result as earliest offset TestUtils.waitForCondition(() -> - Arrays.asList( + List.of( new Row("topic1", 0, 0L), new Row("topic2", 0, 0L), new Row("topic3", 0, 0L), @@ -334,7 +333,7 @@ public class GetOffsetShellTest { // test topics enable remote log storage TestUtils.waitForCondition(() -> - Arrays.asList( + List.of( new Row("topicRLS1", 0, 0L), new Row("topicRLS2", 0, 1L), new Row("topicRLS3", 0, 2L), @@ -353,13 +352,13 @@ public class GetOffsetShellTest { // test topics disable remote log storage // as remote log not enabled, broker return unknown offset for each topic partition and these // unknown offsets are ignored by GetOffsetShell hence we have empty result here. - assertEquals(Collections.emptyList(), + assertEquals(List.of(), executeAndParse("--topic-partitions", "topic\\d+:0", "--time", time)); // test topics enable remote log storage // topicRLS1 has no result because there's no log segments being uploaded to the remote storage TestUtils.waitForCondition(() -> - Arrays.asList( + List.of( new Row("topicRLS2", 0, 0L), new Row("topicRLS3", 0, 1L), new Row("topicRLS4", 0, 2L)) @@ -375,7 +374,7 @@ public class GetOffsetShellTest { String time = String.valueOf(System.currentTimeMillis() / 2); List offsets = executeAndParse("--topic-partitions", "topic.*:0", "--time", time); - List expected = Arrays.asList( + List expected = List.of( new Row("topic1", 0, 0L), new Row("topic2", 0, 0L), new Row("topic3", 0, 0L), @@ -401,7 +400,7 @@ public class GetOffsetShellTest { setUp(); List offsets = executeAndParse("--topic-partitions", "topic1:0,topic2:1,topic(3|4):2,__.*:3", "--exclude-internal-topics"); - List expected = Arrays.asList( + List expected = List.of( new Row("topic1", 0, 1L), new Row("topic2", 1, 2L), new Row("topic3", 2, 3L), @@ -419,7 +418,7 @@ public class GetOffsetShellTest { List offsets = executeAndParse("--topic-partitions", "__.*:0"); - assertEquals(Arrays.asList(new Row("__consumer_offsets", 0, 0L)), offsets); + assertEquals(List.of(new Row("__consumer_offsets", 0, 0L)), offsets); } @ClusterTest @@ -487,9 +486,9 @@ public class GetOffsetShellTest { private List expectedOffsetsWithInternal() { List consOffsets = IntStream.range(0, 4) .mapToObj(i -> new Row("__consumer_offsets", i, 0L)) - .collect(Collectors.toList()); + .toList(); - return Stream.concat(consOffsets.stream(), expectedTestTopicOffsets().stream()).collect(Collectors.toList()); + return Stream.concat(consOffsets.stream(), expectedTestTopicOffsets().stream()).toList(); } private List expectedTestTopicOffsets() { @@ -505,7 +504,7 @@ public class GetOffsetShellTest { private List expectedOffsetsForTopic(int i) { String name = getTopicName(i); - return IntStream.range(0, i).mapToObj(p -> new Row(name, p, (long) i)).collect(Collectors.toList()); + return IntStream.range(0, i).mapToObj(p -> new Row(name, p, (long) i)).toList(); } private List executeAndParse(String... args) { @@ -515,11 +514,11 @@ public class GetOffsetShellTest { .map(i -> i.split(":")) .filter(i -> i.length >= 2) .map(line -> new Row(line[0], Integer.parseInt(line[1]), (line.length == 2 || line[2].isEmpty()) ? null : Long.parseLong(line[2]))) - .collect(Collectors.toList()); + .toList(); } private String[] addBootstrapServer(String... args) { - ArrayList newArgs = new ArrayList<>(Arrays.asList(args)); + ArrayList newArgs = new ArrayList<>(List.of(args)); newArgs.add("--bootstrap-server"); newArgs.add(cluster.bootstrapServers()); diff --git a/tools/src/test/java/org/apache/kafka/tools/JmxToolTest.java b/tools/src/test/java/org/apache/kafka/tools/JmxToolTest.java index 6c93ca69b23..4bc106afc20 100644 --- a/tools/src/test/java/org/apache/kafka/tools/JmxToolTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/JmxToolTest.java @@ -139,9 +139,9 @@ public class JmxToolTest { "--one-time" }; String out = executeAndGetOut(args); - Arrays.stream(out.split("\\r?\\n")).forEach(line -> { - assertTrue(line.matches("([a-zA-Z0-9=:,.]+),\"([ -~]+)\""), line); - }); + Arrays.stream(out.split("\\r?\\n")).forEach(line -> + assertTrue(line.matches("([a-zA-Z0-9=:,.]+),\"([ -~]+)\""), line) + ); } @Test @@ -153,9 +153,9 @@ public class JmxToolTest { "--one-time" }; String out = executeAndGetOut(args); - Arrays.stream(out.split("\\r?\\n")).forEach(line -> { - assertTrue(line.matches("([a-zA-Z0-9=:,.]+)\\t([ -~]+)"), line); - }); + Arrays.stream(out.split("\\r?\\n")).forEach(line -> + assertTrue(line.matches("([a-zA-Z0-9=:,.]+)\\t([ -~]+)"), line) + ); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java index 78f6e9ac156..c3623f24ad0 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java @@ -37,7 +37,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.time.Duration; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -49,8 +48,6 @@ import java.util.concurrent.ExecutionException; import scala.jdk.javaapi.CollectionConverters; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -76,20 +73,20 @@ public class LeaderElectionCommandTest { public void testAllTopicPartition() throws InterruptedException, ExecutionException { String topic = "unclean-topic"; int partition = 0; - List assignment = asList(broker2, broker3); + List assignment = List.of(broker2, broker3); cluster.waitForReadyBrokers(); try (Admin client = cluster.admin()) { - createTopic(client, topic, Collections.singletonMap(partition, assignment)); + createTopic(client, topic, Map.of(partition, assignment)); TopicPartition topicPartition = new TopicPartition(topic, partition); TestUtils.assertLeader(client, topicPartition, broker2); cluster.shutdownBroker(broker3); TestUtils.waitForBrokersOutOfIsr(client, - CollectionConverters.asScala(singletonList(topicPartition)).toSet(), - CollectionConverters.asScala(singletonList(broker3)).toSet() + CollectionConverters.asScala(List.of(topicPartition)).toSet(), + CollectionConverters.asScala(List.of(broker3)).toSet() ); cluster.shutdownBroker(broker2); TestUtils.assertNoLeader(client, topicPartition); @@ -133,11 +130,11 @@ public class LeaderElectionCommandTest { public void testTopicPartition() throws InterruptedException, ExecutionException { String topic = "unclean-topic"; int partition = 0; - List assignment = asList(broker2, broker3); + List assignment = List.of(broker2, broker3); cluster.waitForReadyBrokers(); try (Admin client = cluster.admin()) { - createTopic(client, topic, Collections.singletonMap(partition, assignment)); + createTopic(client, topic, Map.of(partition, assignment)); TopicPartition topicPartition = new TopicPartition(topic, partition); @@ -145,8 +142,8 @@ public class LeaderElectionCommandTest { cluster.shutdownBroker(broker3); TestUtils.waitForBrokersOutOfIsr(client, - CollectionConverters.asScala(singletonList(topicPartition)).toSet(), - CollectionConverters.asScala(singletonList(broker3)).toSet() + CollectionConverters.asScala(List.of(topicPartition)).toSet(), + CollectionConverters.asScala(List.of(broker3)).toSet() ); cluster.shutdownBroker(broker2); TestUtils.assertNoLeader(client, topicPartition); @@ -168,7 +165,7 @@ public class LeaderElectionCommandTest { public void testPathToJsonFile() throws Exception { String topic = "unclean-topic"; int partition = 0; - List assignment = asList(broker2, broker3); + List assignment = List.of(broker2, broker3); cluster.waitForReadyBrokers(); Map> partitionAssignment = new HashMap<>(); @@ -183,15 +180,15 @@ public class LeaderElectionCommandTest { cluster.shutdownBroker(broker3); TestUtils.waitForBrokersOutOfIsr(client, - CollectionConverters.asScala(singletonList(topicPartition)).toSet(), - CollectionConverters.asScala(singletonList(broker3)).toSet() + CollectionConverters.asScala(List.of(topicPartition)).toSet(), + CollectionConverters.asScala(List.of(broker3)).toSet() ); cluster.shutdownBroker(broker2); TestUtils.assertNoLeader(client, topicPartition); cluster.startBroker(broker3); TestUtils.waitForOnlineBroker(client, broker3); - Path topicPartitionPath = tempTopicPartitionFile(singletonList(topicPartition)); + Path topicPartitionPath = tempTopicPartitionFile(List.of(topicPartition)); assertEquals(0, LeaderElectionCommand.mainNoExit( "--bootstrap-server", cluster.bootstrapServers(), @@ -207,7 +204,7 @@ public class LeaderElectionCommandTest { public void testPreferredReplicaElection() throws InterruptedException, ExecutionException { String topic = "preferred-topic"; int partition = 0; - List assignment = asList(broker2, broker3); + List assignment = List.of(broker2, broker3); cluster.waitForReadyBrokers(); try (Admin client = cluster.admin()) { @@ -224,7 +221,7 @@ public class LeaderElectionCommandTest { TestUtils.assertLeader(client, topicPartition, broker3); cluster.startBroker(broker2); TestUtils.waitForBrokersInIsr(client, topicPartition, - CollectionConverters.asScala(singletonList(broker2)).toSet() + CollectionConverters.asScala(List.of(broker2)).toSet() ); assertEquals(0, LeaderElectionCommand.mainNoExit( @@ -254,8 +251,8 @@ public class LeaderElectionCommandTest { String topic = "non-preferred-topic"; int partition0 = 0; int partition1 = 1; - List assignment0 = asList(broker2, broker3); - List assignment1 = asList(broker3, broker2); + List assignment0 = List.of(broker2, broker3); + List assignment1 = List.of(broker3, broker2); cluster.waitForReadyBrokers(); TopicPartition topicPartition0; @@ -277,14 +274,14 @@ public class LeaderElectionCommandTest { TestUtils.assertLeader(client, topicPartition0, broker3); cluster.startBroker(broker2); TestUtils.waitForBrokersInIsr(client, topicPartition0, - CollectionConverters.asScala(singletonList(broker2)).toSet() + CollectionConverters.asScala(List.of(broker2)).toSet() ); TestUtils.waitForBrokersInIsr(client, topicPartition1, - CollectionConverters.asScala(singletonList(broker2)).toSet() + CollectionConverters.asScala(List.of(broker2)).toSet() ); } - Path topicPartitionPath = tempTopicPartitionFile(asList(topicPartition0, topicPartition1)); + Path topicPartitionPath = tempTopicPartitionFile(List.of(topicPartition0, topicPartition1)); String output = ToolsTestUtils.captureStandardOut(() -> LeaderElectionCommand.mainNoExit( "--bootstrap-server", cluster.bootstrapServers(), @@ -308,7 +305,7 @@ public class LeaderElectionCommandTest { private void createTopic(Admin admin, String topic, Map> replicaAssignment) throws ExecutionException, InterruptedException { NewTopic newTopic = new NewTopic(topic, replicaAssignment); - List newTopics = singletonList(newTopic); + List newTopics = List.of(newTopic); CreateTopicsResult createTopicResult = admin.createTopics(newTopics); createTopicResult.all().get(); } diff --git a/tools/src/test/java/org/apache/kafka/tools/LineMessageReaderTest.java b/tools/src/test/java/org/apache/kafka/tools/LineMessageReaderTest.java index 5864869df1a..e6728ccbae2 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LineMessageReaderTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LineMessageReaderTest.java @@ -25,15 +25,12 @@ import org.apache.kafka.tools.api.RecordReader; import org.junit.jupiter.api.Test; import java.io.ByteArrayInputStream; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.Properties; import static java.nio.charset.StandardCharsets.UTF_8; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; import static org.apache.kafka.common.utils.Utils.propsToStringMap; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -57,7 +54,7 @@ public class LineMessageReaderTest { ProducerRecord expected = record( "key0", "value0", - asList( + List.of( new RecordHeader("headerKey0", "headerValue0".getBytes(UTF_8)), new RecordHeader("headerKey1", "headerValue1".getBytes(UTF_8)) ) @@ -67,7 +64,7 @@ public class LineMessageReaderTest { @Test public void testMinimalValidInputWithHeaderKeyAndValue() { - runTest(defaultTestProps(), ":\t\t", record("", "", singletonList(new RecordHeader("", "".getBytes(UTF_8))))); + runTest(defaultTestProps(), ":\t\t", record("", "", List.of(new RecordHeader("", "".getBytes(UTF_8))))); } @Test @@ -90,7 +87,7 @@ public class LineMessageReaderTest { "headerKey0.0::::headerValue0.0---headerKey1.0::::\t\tkey\t\tvalue", record("key", "value", - asList( + List.of( new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)), new RecordHeader("headerKey1.0", "".getBytes(UTF_8)) ) @@ -105,7 +102,7 @@ public class LineMessageReaderTest { Properties props = defaultTestProps(); props.put("parse.key", "false"); - runTest(props, input, record(null, "value", singletonList(new RecordHeader("headerKey", "headerValue".getBytes(UTF_8))))); + runTest(props, input, record(null, "value", List.of(new RecordHeader("headerKey", "headerValue".getBytes(UTF_8))))); } @Test @@ -132,7 +129,7 @@ public class LineMessageReaderTest { ProducerRecord record0 = record( "key0", "value0", - asList( + List.of( new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)), new RecordHeader("headerKey0.1", "headerValue0.1".getBytes(UTF_8)) ) @@ -140,7 +137,7 @@ public class LineMessageReaderTest { ProducerRecord record1 = record( "key1", "value1", - singletonList(new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8))) + List.of(new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8))) ); runTest(props, input, record0, record1); @@ -208,12 +205,12 @@ public class LineMessageReaderTest { props.put("ignore.error", "true"); ProducerRecord validRecord = record("key0", "value0", - singletonList(new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)))); + List.of(new RecordHeader("headerKey0.0", "headerValue0.0".getBytes(UTF_8)))); ProducerRecord missingHeaderDelimiter = record( null, "value1", - asList( + List.of( new RecordHeader("headerKey1.0", "headerValue1.0".getBytes(UTF_8)), new RecordHeader("headerKey1.1", "headerValue1.1[MISSING-HEADER-DELIMITER]key1".getBytes(UTF_8)) ) @@ -222,13 +219,13 @@ public class LineMessageReaderTest { ProducerRecord missingKeyDelimiter = record( null, "key2[MISSING-KEY-DELIMITER]value2", - singletonList(new RecordHeader("headerKey2.0", "headerValue2.0".getBytes(UTF_8))) + List.of(new RecordHeader("headerKey2.0", "headerValue2.0".getBytes(UTF_8))) ); ProducerRecord missingKeyHeaderDelimiter = record( null, "headerKey3.0:headerValue3.0[MISSING-HEADER-DELIMITER]key3[MISSING-KEY-DELIMITER]value3", - Collections.emptyList() + List.of() ); runTest(props, input, validRecord, missingHeaderDelimiter, missingKeyDelimiter, missingKeyHeaderDelimiter); @@ -241,7 +238,7 @@ public class LineMessageReaderTest { Properties props = defaultTestProps(); props.put("ignore.error", "true"); - ProducerRecord expected = record("key0", "value0", singletonList(new RecordHeader("key-val", null))); + ProducerRecord expected = record("key0", "value0", List.of(new RecordHeader("key-val", null))); runTest(props, input, expected); } @@ -288,11 +285,11 @@ public class LineMessageReaderTest { Properties props = defaultTestProps(); props.put("null.marker", ""); runTest(props, input, - record(null, "value", asList(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), + record(null, "value", List.of(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), record("key", null), - record(null, null, asList(new RecordHeader("h0", "".getBytes(UTF_8)), header)), - record("key", null, asList(new RecordHeader("h0", null), header)), - record("key", null, asList(new RecordHeader("h0", null), new RecordHeader("h1", "value".getBytes(UTF_8)))) + record(null, null, List.of(new RecordHeader("h0", "".getBytes(UTF_8)), header)), + record("key", null, List.of(new RecordHeader("h0", null), header)), + record("key", null, List.of(new RecordHeader("h0", null), new RecordHeader("h1", "value".getBytes(UTF_8)))) ); // If the null marker is not set @@ -300,16 +297,16 @@ public class LineMessageReaderTest { props.remove("null.marker"); lineReader.configure(propsToStringMap(props)); Iterator> iter = lineReader.readRecords(new ByteArrayInputStream(input.getBytes())); - assertRecordEquals(record("", "value", asList(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), iter.next()); + assertRecordEquals(record("", "value", List.of(new RecordHeader("h0", "v0".getBytes(UTF_8)), header)), iter.next()); // line 2 is not valid anymore KafkaException expectedException = assertThrows(KafkaException.class, iter::next); assertEquals( "No header key separator found in pair '' on line number 2", expectedException.getMessage() ); - assertRecordEquals(record("", "", asList(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next()); - assertRecordEquals(record("key", "", asList(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next()); - assertRecordEquals(record("key", "", asList( + assertRecordEquals(record("", "", List.of(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next()); + assertRecordEquals(record("key", "", List.of(new RecordHeader("h0", "".getBytes(UTF_8)), header)), iter.next()); + assertRecordEquals(record("key", "", List.of( new RecordHeader("h0", "".getBytes(UTF_8)), new RecordHeader("h1", "value".getBytes(UTF_8)))), iter.next() ); @@ -332,7 +329,7 @@ public class LineMessageReaderTest { // If the null marker is not set props.remove("null.marker"); - runTest(props, input, record("key", "value", asList( + runTest(props, input, record("key", "value", List.of( new RecordHeader("", "v0".getBytes(UTF_8)), new RecordHeader("h1", "v1".getBytes(UTF_8)))) ); diff --git a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java index f4c1f471cc8..80bab74131f 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java @@ -31,8 +31,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -53,7 +51,7 @@ public class LogDirsCommandTest { @ClusterTest(brokers = 3) public void testLogDirsWithoutBrokers(ClusterInstance clusterInstance) { createTopic(clusterInstance, TOPIC); - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe"), admin)); // check all brokers are present @@ -73,7 +71,7 @@ public class LogDirsCommandTest { @ClusterTest(brokers = 3) public void testLogDirsWithBrokers(ClusterInstance clusterInstance) { createTopic(clusterInstance, TOPIC); - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { int brokerId = 0; String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--broker-list", String.valueOf(brokerId), "--describe"), admin)); @@ -82,7 +80,7 @@ public class LogDirsCommandTest { clusterInstance.brokerIds().stream().filter(id -> id != brokerId).forEach(id -> assertFalse(output.contains("\"broker\":" + id))); // check log dir and topic partition are present - Map> logDirs = assertDoesNotThrow(() -> admin.describeLogDirs(Collections.singleton(brokerId)).allDescriptions().get()); + Map> logDirs = assertDoesNotThrow(() -> admin.describeLogDirs(Set.of(brokerId)).allDescriptions().get()); assertEquals(1, logDirs.size()); logDirs.forEach((brokerIdValue, logDirInfo) -> { assertFalse(logDirInfo.isEmpty()); @@ -98,7 +96,7 @@ public class LogDirsCommandTest { @ClusterTest public void testLogDirsWithNonExistentTopic(ClusterInstance clusterInstance) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--topic-list", TOPIC, "--describe"), admin)); // check all brokers are present clusterInstance.brokerIds().forEach(brokerId -> assertTrue(output.contains("\"broker\":" + brokerId))); @@ -109,9 +107,9 @@ public class LogDirsCommandTest { logDirs.forEach((brokerId, logDirInfo) -> logDirInfo.forEach((logDir, logDirInfoValue) -> { assertTrue(output.contains("\"logDir\":\"" + logDir + "\"")); - logDirInfoValue.replicaInfos().forEach((topicPartition, replicaInfo) -> { - assertFalse(output.contains("\"partition\":\"" + topicPartition + "\"")); - }); + logDirInfoValue.replicaInfos().forEach((topicPartition, replicaInfo) -> + assertFalse(output.contains("\"partition\":\"" + topicPartition + "\"")) + ); })); } } @@ -120,7 +118,7 @@ public class LogDirsCommandTest { public void testLogDirsWithSpecificTopic(ClusterInstance clusterInstance) { createTopic(clusterInstance, TOPIC); createTopic(clusterInstance, "other-topic"); - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { String output = assertDoesNotThrow(() -> execute(fromArgsToOptions("--bootstrap-server", clusterInstance.bootstrapServers(), "--topic-list", TOPIC, "--describe"), admin)); // check all brokers are present clusterInstance.brokerIds().forEach(brokerId -> assertTrue(output.contains("\"broker\":" + brokerId))); @@ -133,9 +131,9 @@ public class LogDirsCommandTest { logDirs.forEach((brokerId, logDirInfo) -> logDirInfo.forEach((logDir, logDirInfoValue) -> { assertTrue(output.contains("\"logDir\":\"" + logDir + "\"")); - logDirInfoValue.replicaInfos().keySet().stream().filter(tp -> !tp.topic().equals(TOPIC)).forEach(tp -> { - assertFalse(output.contains("\"partition\":\"" + tp + "\"")); - }); + logDirInfoValue.replicaInfos().keySet().stream().filter(tp -> !tp.topic().equals(TOPIC)).forEach(tp -> + assertFalse(output.contains("\"partition\":\"" + tp + "\"")) + ); })); } } @@ -143,7 +141,7 @@ public class LogDirsCommandTest { @Test public void shouldThrowWhenQueryingNonExistentBrokers() { Node broker = new Node(1, "hostname", 9092); - try (MockAdminClient adminClient = new MockAdminClient(Collections.singletonList(broker), broker)) { + try (MockAdminClient adminClient = new MockAdminClient(List.of(broker), broker)) { RuntimeException exception = assertThrows(RuntimeException.class, () -> execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "0,1,2", "--describe"), adminClient)); assertNotNull(exception.getCause()); assertEquals(TerseException.class, exception.getCause().getClass()); @@ -155,7 +153,7 @@ public class LogDirsCommandTest { @SuppressWarnings("unchecked") public void shouldNotThrowWhenDuplicatedBrokers() throws JsonProcessingException { Node broker = new Node(1, "hostname", 9092); - try (MockAdminClient adminClient = new MockAdminClient(Collections.singletonList(broker), broker)) { + try (MockAdminClient adminClient = new MockAdminClient(List.of(broker), broker)) { String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "1,1", "--describe"), adminClient); String[] standardOutputLines = standardOutput.split("\n"); assertEquals(3, standardOutputLines.length); @@ -172,13 +170,13 @@ public class LogDirsCommandTest { public void shouldQueryAllBrokersIfNonSpecified() throws JsonProcessingException { Node brokerOne = new Node(1, "hostname", 9092); Node brokerTwo = new Node(2, "hostname", 9092); - try (MockAdminClient adminClient = new MockAdminClient(Arrays.asList(brokerTwo, brokerOne), brokerOne)) { + try (MockAdminClient adminClient = new MockAdminClient(List.of(brokerTwo, brokerOne), brokerOne)) { String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--describe"), adminClient); String[] standardOutputLines = standardOutput.split("\n"); assertEquals(3, standardOutputLines.length); Map information = new ObjectMapper().readValue(standardOutputLines[2], HashMap.class); List brokersInformation = (List) information.get("brokers"); - Set brokerIds = new HashSet() {{ + Set brokerIds = new HashSet<>() {{ add((Integer) ((HashMap) brokersInformation.get(0)).get("broker")); add((Integer) ((HashMap) brokersInformation.get(1)).get("broker")); }}; @@ -192,7 +190,7 @@ public class LogDirsCommandTest { public void shouldQuerySpecifiedBroker() throws JsonProcessingException { Node brokerOne = new Node(1, "hostname", 9092); Node brokerTwo = new Node(2, "hostname", 9092); - try (MockAdminClient adminClient = new MockAdminClient(Arrays.asList(brokerOne, brokerTwo), brokerOne)) { + try (MockAdminClient adminClient = new MockAdminClient(List.of(brokerOne, brokerTwo), brokerOne)) { String standardOutput = execute(fromArgsToOptions("--bootstrap-server", "EMPTY", "--broker-list", "1", "--describe"), adminClient); String[] standardOutputLines = standardOutput.split("\n"); assertEquals(3, standardOutputLines.length); @@ -220,8 +218,8 @@ public class LogDirsCommandTest { } private void createTopic(ClusterInstance clusterInstance, String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, Collections.singletonMap(0, Collections.singletonList(0))))).topicId(topic).get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + assertDoesNotThrow(() -> admin.createTopics(List.of(new NewTopic(topic, Map.of(0, List.of(0))))).topicId(topic).get()); assertDoesNotThrow(() -> clusterInstance.waitTopicCreation(topic, 1)); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java index 5c8338d90ca..621943f695a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.regex.Pattern; -import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -54,7 +53,7 @@ class MetadataQuorumCommandTest { MetadataQuorumCommand.mainNoExit("--bootstrap-server", cluster.bootstrapServers(), "describe", "--replication") ); - List outputs = Arrays.stream(describeOutput.split("\n")).collect(Collectors.toList()); + List outputs = Arrays.stream(describeOutput.split("\n")).toList(); String header = outputs.get(0); List data = outputs.subList(1, outputs.size()); diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandUnitTest.java index df88ad2cbc8..b3131ec3b59 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandUnitTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandUnitTest.java @@ -28,7 +28,6 @@ import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.List; import java.util.Optional; import java.util.Properties; @@ -41,7 +40,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class MetadataQuorumCommandUnitTest { @Test public void testRemoveControllerDryRun() { - List outputs = Arrays.asList( + List outputs = List.of( ToolsTestUtils.captureStandardOut(() -> assertEquals(0, MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "remove-controller", @@ -250,7 +249,7 @@ public class MetadataQuorumCommandUnitTest { new MetadataQuorumCommandUnitTestEnv(Optional. of(Uuid.fromString("wZoXPqWoSu6F6c8MkmdyAg")))) { File propsFile = testEnv.writePropertiesFile(); - List outputs = Arrays.asList( + List outputs = List.of( ToolsTestUtils.captureStandardOut(() -> assertEquals(0, MetadataQuorumCommand.mainNoExit("--bootstrap-server", "localhost:9092", "--command-config", propsFile.getAbsolutePath(), diff --git a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java index 0ec9c733164..7217c2101a0 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ProducerPerformanceTest.java @@ -38,7 +38,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Properties; import java.util.SplittableRandom; @@ -94,7 +93,7 @@ public class ProducerPerformanceTest { @Test public void testReadProps() throws Exception { - List producerProps = Collections.singletonList("bootstrap.servers=localhost:9000"); + List producerProps = List.of("bootstrap.servers=localhost:9000"); File producerConfig = createTempFile("acks=1"); Properties prop = ProducerPerformance.readProps(producerProps, producerConfig.getAbsolutePath()); @@ -360,7 +359,7 @@ public class ProducerPerformanceTest { @Test public void testClientIdOverride() throws Exception { - List producerProps = Collections.singletonList("client.id=producer-1"); + List producerProps = List.of("client.id=producer-1"); Properties prop = ProducerPerformance.readProps(producerProps, null); @@ -370,7 +369,7 @@ public class ProducerPerformanceTest { @Test public void testDefaultClientId() throws Exception { - List producerProps = Collections.singletonList("acks=1"); + List producerProps = List.of("acks=1"); Properties prop = ProducerPerformance.readProps(producerProps, null); @@ -391,9 +390,7 @@ public class ProducerPerformanceTest { ProducerPerformance.Stats stats = new ProducerPerformance.Stats(numRecords, false); for (long i = 0; i < numRecords; i++) { final Callback callback = new ProducerPerformance.PerfCallback(0, 100, stats, null); - CompletableFuture.runAsync(() -> { - callback.onCompletion(null, null); - }, singleThreaded); + CompletableFuture.runAsync(() -> callback.onCompletion(null, null), singleThreaded); } singleThreaded.shutdown(); diff --git a/tools/src/test/java/org/apache/kafka/tools/PushHttpMetricsReporterTest.java b/tools/src/test/java/org/apache/kafka/tools/PushHttpMetricsReporterTest.java index c37e27bf3d9..00215d6c7ee 100644 --- a/tools/src/test/java/org/apache/kafka/tools/PushHttpMetricsReporterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/PushHttpMetricsReporterTest.java @@ -43,8 +43,7 @@ import java.net.InetAddress; import java.net.MalformedURLException; import java.net.URL; import java.net.UnknownHostException; -import java.util.Arrays; -import java.util.Collections; +import java.util.ArrayList; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -186,41 +185,41 @@ public class PushHttpMetricsReporterTest { verifyConfigure(); KafkaMetric metric1 = new KafkaMetric( new Object(), - new MetricName("name1", "group1", "desc1", Collections.singletonMap("key1", "value1")), + new MetricName("name1", "group1", "desc1", Map.of("key1", "value1")), new ImmutableValue<>(1.0), null, time ); KafkaMetric newMetric1 = new KafkaMetric( new Object(), - new MetricName("name1", "group1", "desc1", Collections.singletonMap("key1", "value1")), + new MetricName("name1", "group1", "desc1", Map.of("key1", "value1")), new ImmutableValue<>(-1.0), null, time ); KafkaMetric metric2 = new KafkaMetric( new Object(), - new MetricName("name2", "group2", "desc2", Collections.singletonMap("key2", "value2")), + new MetricName("name2", "group2", "desc2", Map.of("key2", "value2")), new ImmutableValue<>(2.0), null, time ); KafkaMetric metric3 = new KafkaMetric( new Object(), - new MetricName("name3", "group3", "desc3", Collections.singletonMap("key3", "value3")), + new MetricName("name3", "group3", "desc3", Map.of("key3", "value3")), new ImmutableValue<>(3.0), null, time ); KafkaMetric metric4 = new KafkaMetric( new Object(), - new MetricName("name4", "group4", "desc4", Collections.singletonMap("key4", "value4")), + new MetricName("name4", "group4", "desc4", Map.of("key4", "value4")), new ImmutableValue<>("value4"), null, time ); - reporter.init(Arrays.asList(metric1, metric2, metric4)); + reporter.init(List.of(metric1, metric2, metric4)); reporter.metricChange(newMetric1); // added in init, modified reporter.metricChange(metric3); // added by change reporter.metricRemoval(metric2); // added in init, deleted by removal @@ -236,7 +235,7 @@ public class PushHttpMetricsReporterTest { JsonNode metrics = payload.get("metrics"); assertTrue(metrics.isArray()); assertEquals(3, metrics.size()); - List metricsList = Arrays.asList(metrics.get(0), metrics.get(1), metrics.get(2)); + List metricsList = new ArrayList<>(List.of(metrics.get(0), metrics.get(1), metrics.get(2))); // Sort metrics based on name so that we can verify the value for each metric below metricsList.sort(Comparator.comparing(m -> m.get("name").textValue())); diff --git a/tools/src/test/java/org/apache/kafka/tools/ReplicaVerificationToolTest.java b/tools/src/test/java/org/apache/kafka/tools/ReplicaVerificationToolTest.java index 14fbbe4bcae..68e64aa9c73 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ReplicaVerificationToolTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ReplicaVerificationToolTest.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.record.SimpleRecord; import org.junit.jupiter.api.Test; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.stream.IntStream; @@ -43,8 +42,8 @@ public class ReplicaVerificationToolTest { }}; ReplicaVerificationTool.ReplicaBuffer replicaBuffer = - new ReplicaVerificationTool.ReplicaBuffer(expectedReplicasPerTopicAndPartition, Collections.emptyMap(), 2, 0); - expectedReplicasPerTopicAndPartition.forEach((tp, numReplicas) -> { + new ReplicaVerificationTool.ReplicaBuffer(expectedReplicasPerTopicAndPartition, Map.of(), 2, 0); + expectedReplicasPerTopicAndPartition.forEach((tp, numReplicas) -> IntStream.range(0, numReplicas).forEach(replicaId -> { SimpleRecord[] records = IntStream.rangeClosed(0, 5) .mapToObj(index -> new SimpleRecord(("key " + index).getBytes(), ("value " + index).getBytes())) @@ -60,8 +59,8 @@ public class ReplicaVerificationToolTest { .setRecords(memoryRecords); replicaBuffer.addFetchedData(tp, replicaId, partitionData); - }); - }); + }) + ); replicaBuffer.verifyCheckSum(line -> sb.append(format("%s%n", line))); String output = sb.toString().trim(); diff --git a/tools/src/test/java/org/apache/kafka/tools/StreamsResetterTest.java b/tools/src/test/java/org/apache/kafka/tools/StreamsResetterTest.java index 2425c21fe90..ef8875f7c49 100644 --- a/tools/src/test/java/org/apache/kafka/tools/StreamsResetterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/StreamsResetterTest.java @@ -33,9 +33,9 @@ import org.junit.jupiter.api.Timeout; import java.time.Duration; import java.time.Instant; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutionException; @@ -53,7 +53,7 @@ public class StreamsResetterTest { @BeforeEach public void beforeEach() { - consumer.assign(Collections.singletonList(topicPartition)); + consumer.assign(List.of(topicPartition)); consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 0L, new byte[] {}, new byte[] {})); consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 1L, new byte[] {}, new byte[] {})); consumer.addRecord(new ConsumerRecord<>(TOPIC, 0, 2L, new byte[] {}, new byte[] {})); @@ -82,7 +82,7 @@ public class StreamsResetterTest { final long beginningOffset = 5L; final long endOffset = 10L; final MockConsumer emptyConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()); - emptyConsumer.assign(Collections.singletonList(topicPartition)); + emptyConsumer.assign(List.of(topicPartition)); final Map beginningOffsetsMap = new HashMap<>(); beginningOffsetsMap.put(topicPartition, beginningOffset); @@ -255,10 +255,10 @@ public class StreamsResetterTest { public void shouldDeleteTopic() throws InterruptedException, ExecutionException { final Cluster cluster = createCluster(1); try (final MockAdminClient adminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - final TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); - adminClient.addTopic(false, TOPIC, Collections.singletonList(topicPartitionInfo), null); - streamsResetter.doDelete(Collections.singletonList(TOPIC), adminClient); - assertEquals(Collections.emptySet(), adminClient.listTopics().names().get()); + final TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), List.of()); + adminClient.addTopic(false, TOPIC, List.of(topicPartitionInfo), null); + streamsResetter.doDelete(List.of(TOPIC), adminClient); + assertEquals(Set.of(), adminClient.listTopics().names().get()); } } @@ -274,7 +274,7 @@ public class StreamsResetterTest { public void testResetToDatetimeWhenPartitionIsEmptyResetsToLatestOffset() { final long beginningAndEndOffset = 5L; // Empty partition implies beginning offset == end offset final MockConsumer emptyConsumer = new EmptyPartitionConsumer<>(AutoOffsetResetStrategy.EARLIEST.name()); - emptyConsumer.assign(Collections.singletonList(topicPartition)); + emptyConsumer.assign(List.of(topicPartition)); final Map beginningOffsetsMap = new HashMap<>(); beginningOffsetsMap.put(topicPartition, beginningAndEndOffset); @@ -299,8 +299,8 @@ public class StreamsResetterTest { nodes.put(i, new Node(i, "localhost", 8121 + i)); } return new Cluster("mockClusterId", nodes.values(), - Collections.emptySet(), Collections.emptySet(), - Collections.emptySet(), nodes.get(0)); + Set.of(), Set.of(), + Set.of(), nodes.get(0)); } private static class EmptyPartitionConsumer extends MockConsumer { diff --git a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java index f279c3e99f8..0886fb3d15e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/ToolsTestUtils.java @@ -30,7 +30,6 @@ import java.io.IOException; import java.io.PrintStream; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.HashMap; import java.util.List; @@ -146,7 +145,7 @@ public class ToolsTestUtils { public static void removePartitionReplicaThrottles(Admin adminClient, Set partitions) throws ExecutionException, InterruptedException { Map> throttles = partitions.stream().collect(Collectors.toMap( tp -> new ConfigResource(ConfigResource.Type.TOPIC, tp.topic()), - tp -> Arrays.asList( + tp -> List.of( new AlterConfigOp(new ConfigEntry(QuotaConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), AlterConfigOp.OpType.DELETE), new AlterConfigOp(new ConfigEntry(QuotaConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, ""), diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index 99e5e7be9c7..fe9028b8c41 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -110,7 +110,7 @@ public class TopicCommandTest { @Test public void testIsNotUnderReplicatedWhenAdding() { - List replicaIds = Arrays.asList(1, 2); + List replicaIds = List.of(1, 2); List replicas = new ArrayList<>(); for (int id : replicaIds) { replicas.add(new Node(id, "localhost", 9090 + id)); @@ -118,9 +118,9 @@ public class TopicCommandTest { TopicCommand.PartitionDescription partitionDescription = new TopicCommand.PartitionDescription("test-topic", new TopicPartitionInfo(0, new Node(1, "localhost", 9091), replicas, - Collections.singletonList(new Node(1, "localhost", 9091))), + List.of(new Node(1, "localhost", 9091))), null, false, - new PartitionReassignment(replicaIds, Arrays.asList(2), Collections.emptyList()) + new PartitionReassignment(replicaIds, List.of(2), List.of()) ); assertFalse(partitionDescription.isUnderReplicated()); @@ -236,9 +236,9 @@ public class TopicCommandTest { public void testParseAssignment() { Map> actualAssignment = TopicCommand.parseReplicaAssignment("5:4,3:2,1:0"); Map> expectedAssignment = new HashMap<>(); - expectedAssignment.put(0, Arrays.asList(5, 4)); - expectedAssignment.put(1, Arrays.asList(3, 2)); - expectedAssignment.put(2, Arrays.asList(1, 0)); + expectedAssignment.put(0, List.of(5, 4)); + expectedAssignment.put(1, List.of(3, 2)); + expectedAssignment.put(2, List.of(1, 0)); assertEquals(expectedAssignment, actualAssignment); } @@ -257,7 +257,7 @@ public class TopicCommandTest { }))); NewTopic expectedNewTopic = new NewTopic(topicName, Optional.empty(), Optional.empty()) - .configs(Collections.emptyMap()); + .configs(Map.of()); verify(adminClient, times(1)).createTopics( eq(Set.of(expectedNewTopic)), @@ -285,7 +285,7 @@ public class TopicCommandTest { assertInstanceOf(ThrottlingQuotaExceededException.class, exception.getCause()); verify(adminClient).deleteTopics( - argThat((Collection topics) -> topics.equals(Arrays.asList(topicName))), + argThat((Collection topics) -> topics.equals(List.of(topicName))), argThat((DeleteTopicsOptions options) -> !options.shouldRetryOnQuotaViolation())); } @@ -298,9 +298,9 @@ public class TopicCommandTest { when(adminClient.listTopics(any())).thenReturn(listResult); TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, new Node(0, "", 0), - Collections.emptyList(), Collections.emptyList()); + List.of(), List.of()); DescribeTopicsResult describeResult = AdminClientTestUtils.describeTopicsResult(topicName, - new TopicDescription(topicName, false, Collections.singletonList(topicPartitionInfo))); + new TopicDescription(topicName, false, List.of(topicPartitionInfo))); when(adminClient.describeTopics(anyCollection())).thenReturn(describeResult); CreatePartitionsResult result = AdminClientTestUtils.createPartitionsResult(topicName, Errors.THROTTLING_QUOTA_EXCEEDED.exception()); @@ -365,7 +365,7 @@ public class TopicCommandTest { rackInfo.put(4, infoPerBroker5); rackInfo.put(5, infoPerBroker6); - return Collections.singletonList(ClusterConfig.defaultBuilder() + return List.of(ClusterConfig.defaultBuilder() .setBrokers(6) .setServerProperties(serverProp) .setPerServerProperties(rackInfo) @@ -385,13 +385,13 @@ public class TopicCommandTest { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); Assertions.assertTrue(adminClient.listTopics().names().get().contains(testTopicName), "Admin client didn't see the created topic. It saw: " + adminClient.listTopics().names().get()); - adminClient.deleteTopics(Collections.singletonList(testTopicName)); + adminClient.deleteTopics(List.of(testTopicName)); clusterInstance.waitTopicDeletion(testTopicName); Assertions.assertTrue(adminClient.listTopics().names().get().isEmpty(), "Admin client see the created topic. It saw: " + adminClient.listTopics().names().get()); @@ -409,14 +409,14 @@ public class TopicCommandTest { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); Assertions.assertTrue(adminClient.listTopics().names().get().contains(testTopicName), "Admin client didn't see the created topic. It saw: " + adminClient.listTopics().names().get()); List partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) + .describeTopics(List.of(testTopicName)) .allTopicNames() .get() .get(testTopicName) @@ -424,7 +424,7 @@ public class TopicCommandTest { Assertions.assertEquals(defaultNumPartitions, partitions.size(), "Unequal partition size: " + partitions.size()); Assertions.assertEquals(defaultReplicationFactor, (short) partitions.get(0).replicas().size(), "Unequal replication factor: " + partitions.get(0).replicas().size()); - adminClient.deleteTopics(Collections.singletonList(testTopicName)); + adminClient.deleteTopics(List.of(testTopicName)); clusterInstance.waitTopicDeletion(testTopicName); Assertions.assertTrue(adminClient.listTopics().names().get().isEmpty(), "Admin client see the created topic. It saw: " + adminClient.listTopics().names().get()); @@ -442,10 +442,10 @@ public class TopicCommandTest { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, 2, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, 2, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, 2); List partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) + .describeTopics(List.of(testTopicName)) .allTopicNames() .get() .get(testTopicName) @@ -460,10 +460,10 @@ public class TopicCommandTest { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, (short) 2))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, (short) 2))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); List partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) + .describeTopics(List.of(testTopicName)) .allTopicNames() .get() .get(testTopicName) @@ -483,11 +483,11 @@ public class TopicCommandTest { Map topicConfig = new HashMap<>(); topicConfig.put(TopicConfig.DELETE_RETENTION_MS_CONFIG, "1000"); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, 2, (short) 2).configs(topicConfig))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, 2, (short) 2).configs(topicConfig))); clusterInstance.waitTopicCreation(testTopicName, 2); - Config configs = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource); + Config configs = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource); assertEquals(1000, Integer.valueOf(configs.get("delete.retention.ms").value()), "Config not set correctly: " + configs.get("delete.retention.ms").value()); } @@ -502,7 +502,7 @@ public class TopicCommandTest { clusterInstance, "--create", "--partitions", Integer.toString(defaultNumPartitions), "--replication-factor", "1", "--topic", testTopicName); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); // try to re-create the topic @@ -516,7 +516,7 @@ public class TopicCommandTest { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin(); TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); TopicCommand.TopicCommandOptions createOpts = @@ -526,7 +526,7 @@ public class TopicCommandTest { } private List getPartitionReplicas(List partitions, int partitionNumber) { - return partitions.get(partitionNumber).replicas().stream().map(Node::id).collect(Collectors.toList()); + return partitions.get(partitionNumber).replicas().stream().map(Node::id).toList(); } @ClusterTemplate("generate") @@ -535,15 +535,15 @@ public class TopicCommandTest { try (Admin adminClient = clusterInstance.admin()) { String testTopicName = TestUtils.randomString(10); - replicaAssignmentMap.put(0, Arrays.asList(5, 4)); - replicaAssignmentMap.put(1, Arrays.asList(3, 2)); - replicaAssignmentMap.put(2, Arrays.asList(1, 0)); + replicaAssignmentMap.put(0, List.of(5, 4)); + replicaAssignmentMap.put(1, List.of(3, 2)); + replicaAssignmentMap.put(2, List.of(1, 0)); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, replicaAssignmentMap))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, replicaAssignmentMap))); clusterInstance.waitTopicCreation(testTopicName, 3); List partitions = adminClient - .describeTopics(Collections.singletonList(testTopicName)) + .describeTopics(List.of(testTopicName)) .allTopicNames() .get() .get(testTopicName) @@ -551,11 +551,11 @@ public class TopicCommandTest { assertEquals(3, partitions.size(), "Unequal partition size: " + partitions.size()); - assertEquals(Arrays.asList(5, 4), getPartitionReplicas(partitions, 0), + assertEquals(List.of(5, 4), getPartitionReplicas(partitions, 0), "Unexpected replica assignment: " + getPartitionReplicas(partitions, 0)); - assertEquals(Arrays.asList(3, 2), getPartitionReplicas(partitions, 1), + assertEquals(List.of(3, 2), getPartitionReplicas(partitions, 1), "Unexpected replica assignment: " + getPartitionReplicas(partitions, 1)); - assertEquals(Arrays.asList(1, 0), getPartitionReplicas(partitions, 2), + assertEquals(List.of(1, 0), getPartitionReplicas(partitions, 2), "Unexpected replica assignment: " + getPartitionReplicas(partitions, 2)); } } @@ -610,7 +610,7 @@ public class TopicCommandTest { public void testListTopics(ClusterInstance clusterInstance) throws InterruptedException { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); String output = captureListTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--list")); @@ -626,9 +626,9 @@ public class TopicCommandTest { String topic3 = "oooof.testTopic1"; int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(topic1, partition, replicationFactor))); - adminClient.createTopics(Collections.singletonList(new NewTopic(topic2, partition, replicationFactor))); - adminClient.createTopics(Collections.singletonList(new NewTopic(topic3, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topic1, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topic2, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topic3, partition, replicationFactor))); clusterInstance.waitTopicCreation(topic1, partition); clusterInstance.waitTopicCreation(topic2, partition); clusterInstance.waitTopicCreation(topic3, partition); @@ -647,7 +647,7 @@ public class TopicCommandTest { String hiddenConsumerTopic = Topic.GROUP_METADATA_TOPIC_NAME; int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(topic1, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topic1, partition, replicationFactor))); clusterInstance.waitTopicCreation(topic1, partition); String output = captureListTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--list", "--exclude-internal")); @@ -663,7 +663,7 @@ public class TopicCommandTest { TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partition); topicService.alterTopic(buildTopicCommandOptionsWithBootstrap(clusterInstance, "--alter", "--topic", testTopicName, "--partitions", "3")); @@ -676,7 +676,7 @@ public class TopicCommandTest { () -> clusterInstance.brokers().values().stream().allMatch( b -> b.metadataCache().numPartitions(testTopicName).orElse(0) == 3), TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker"); - TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get(); + TopicDescription topicDescription = adminClient.describeTopics(List.of(testTopicName)).topicNameValues().get(testTopicName).get(); assertEquals(3, topicDescription.partitions().size(), "Expected partition count to be 3. Got: " + topicDescription.partitions().size()); } } @@ -689,7 +689,7 @@ public class TopicCommandTest { int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partition); topicService.alterTopic(buildTopicCommandOptionsWithBootstrap(clusterInstance, "--alter", @@ -705,10 +705,10 @@ public class TopicCommandTest { b -> b.metadataCache().numPartitions(testTopicName).orElse(0) == 3), TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker"); - TopicDescription topicDescription = adminClient.describeTopics(Collections.singletonList(testTopicName)).topicNameValues().get(testTopicName).get(); + TopicDescription topicDescription = adminClient.describeTopics(List.of(testTopicName)).topicNameValues().get(testTopicName).get(); assertEquals(3, topicDescription.partitions().size(), "Expected partition count to be 3. Got: " + topicDescription.partitions().size()); List partitionReplicas = getPartitionReplicas(topicDescription.partitions(), 2); - assertEquals(Arrays.asList(4, 2), partitionReplicas, "Expected to have replicas 4,2. Got: " + partitionReplicas); + assertEquals(List.of(4, 2), partitionReplicas, "Expected to have replicas 4,2. Got: " + partitionReplicas); } } @@ -721,7 +721,7 @@ public class TopicCommandTest { int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partition); assertThrows(ExecutionException.class, @@ -739,7 +739,7 @@ public class TopicCommandTest { TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partition); assertThrows(ExecutionException.class, @@ -756,7 +756,7 @@ public class TopicCommandTest { try (Admin adminClient = clusterInstance.admin(); TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); assertThrows(ExecutionException.class, @@ -805,15 +805,15 @@ public class TopicCommandTest { int numPartitions = 18; int replicationFactor = 3; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, numPartitions, (short) replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, numPartitions, (short) replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, numPartitions); - Map> assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) + Map> assignment = adminClient.describeTopics(List.of(testTopicName)) .allTopicNames().get().get(testTopicName).partitions() .stream() .collect(Collectors.toMap( - info -> info.partition(), - info -> info.replicas().stream().map(Node::id).collect(Collectors.toList()))); + TopicPartitionInfo::partition, + info -> info.replicas().stream().map(Node::id).toList())); checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), numPartitions, replicationFactor, true, true, true); @@ -832,9 +832,9 @@ public class TopicCommandTest { () -> clusterInstance.brokers().values().stream().allMatch(p -> p.metadataCache().numPartitions(testTopicName).orElse(0) == alteredNumPartitions), TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker"); - assignment = adminClient.describeTopics(Collections.singletonList(testTopicName)) + assignment = adminClient.describeTopics(List.of(testTopicName)) .allTopicNames().get().get(testTopicName).partitions().stream() - .collect(Collectors.toMap(info -> info.partition(), info -> info.replicas().stream().map(Node::id).collect(Collectors.toList()))); + .collect(Collectors.toMap(TopicPartitionInfo::partition, info -> info.replicas().stream().map(Node::id).toList())); checkReplicaDistribution(assignment, rackInfo, rackInfo.size(), alteredNumPartitions, replicationFactor, true, true, true); @@ -850,11 +850,11 @@ public class TopicCommandTest { String cleanUpPolicy = "compact"; HashMap topicConfig = new HashMap<>(); topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, cleanUpPolicy); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor).configs(topicConfig))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor).configs(topicConfig))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); ConfigResource configResource = new ConfigResource(ConfigResource.Type.TOPIC, testTopicName); - Config props = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource); + Config props = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource); assertNotNull(props.get(TopicConfig.CLEANUP_POLICY_CONFIG), "Properties after creation don't contain " + cleanUpPolicy); assertEquals(cleanUpPolicy, props.get(TopicConfig.CLEANUP_POLICY_CONFIG).value(), "Properties after creation have incorrect value"); @@ -868,7 +868,7 @@ public class TopicCommandTest { () -> clusterInstance.brokers().values().stream().allMatch(p -> p.metadataCache().numPartitions(testTopicName).orElse(0) == numPartitionsModified), TestUtils.DEFAULT_MAX_WAIT_MS, "Timeout waiting for new assignment propagating to broker"); - Config newProps = adminClient.describeConfigs(Collections.singleton(configResource)).all().get().get(configResource); + Config newProps = adminClient.describeConfigs(Set.of(configResource)).all().get().get(configResource); assertNotNull(newProps.get(TopicConfig.CLEANUP_POLICY_CONFIG), "Updated properties do not contain " + TopicConfig.CLEANUP_POLICY_CONFIG); assertEquals(cleanUpPolicy, newProps.get(TopicConfig.CLEANUP_POLICY_CONFIG).value(), "Updated properties have incorrect value"); @@ -887,7 +887,7 @@ public class TopicCommandTest { TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { String testTopicName = TestUtils.randomString(10); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); // delete the NormalTopic @@ -913,7 +913,7 @@ public class TopicCommandTest { TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { // create the topic with colliding chars String topicWithCollidingChar = "test.a"; - adminClient.createTopics(Collections.singletonList(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(topicWithCollidingChar, defaultNumPartitions); // delete the topic @@ -927,7 +927,7 @@ public class TopicCommandTest { clusterInstance.waitTopicDeletion(topicWithCollidingChar); // recreate same topic - adminClient.createTopics(Collections.singletonList(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topicWithCollidingChar, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(topicWithCollidingChar, defaultNumPartitions); } } @@ -944,7 +944,7 @@ public class TopicCommandTest { TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { // create the offset topic - adminClient.createTopics(Collections.singletonList(new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(Topic.GROUP_METADATA_TOPIC_NAME, defaultNumPartitions); // Try to delete the Topic.GROUP_METADATA_TOPIC_NAME which is allowed by default. @@ -1001,7 +1001,7 @@ public class TopicCommandTest { try (Admin adminClient = clusterInstance.admin()) { int partition = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partition, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partition, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partition); String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe", "--topic", testTopicName)); @@ -1075,7 +1075,7 @@ public class TopicCommandTest { int partitions = 3; short replicationFactor = 1; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partitions); // check which partition is on broker 0 which we'll kill @@ -1102,11 +1102,11 @@ public class TopicCommandTest { try (Admin adminClient = clusterInstance.admin()) { int partitions = 1; short replicationFactor = 3; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor))); clusterInstance.waitTopicCreation(testTopicName, partitions); clusterInstance.shutdownBroker(0); - Assertions.assertEquals(clusterInstance.aliveBrokers().size(), 2); + Assertions.assertEquals(2, clusterInstance.aliveBrokers().size()); TestUtils.waitForCondition( () -> clusterInstance.aliveBrokers().values().stream().allMatch( @@ -1133,7 +1133,7 @@ public class TopicCommandTest { topicConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "3"); int partitions = 1; short replicationFactor = 3; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); clusterInstance.waitTopicCreation(testTopicName, partitions); clusterInstance.shutdownBroker(0); @@ -1157,7 +1157,7 @@ public class TopicCommandTest { try (Admin adminClient = clusterInstance.admin(); KafkaProducer producer = createProducer(clusterInstance)) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); TopicPartition tp = new TopicPartition(testTopicName, 0); @@ -1170,22 +1170,22 @@ public class TopicCommandTest { // throughput so the reassignment doesn't complete quickly. List brokerIds = new ArrayList<>(clusterInstance.brokerIds()); - ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp), 1); + ToolsTestUtils.setReplicationThrottleForPartitions(adminClient, brokerIds, Set.of(tp), 1); - TopicDescription testTopicDesc = adminClient.describeTopics(Collections.singleton(testTopicName)).allTopicNames().get().get(testTopicName); + TopicDescription testTopicDesc = adminClient.describeTopics(Set.of(testTopicName)).allTopicNames().get().get(testTopicName); TopicPartitionInfo firstPartition = testTopicDesc.partitions().get(0); - List replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).collect(Collectors.toList()); + List replicasOfFirstPartition = firstPartition.replicas().stream().map(Node::id).toList(); List replicasDiff = new ArrayList<>(brokerIds); replicasDiff.removeAll(replicasOfFirstPartition); Integer targetReplica = replicasDiff.get(0); - adminClient.alterPartitionReassignments(Collections.singletonMap(tp, - Optional.of(new NewPartitionReassignment(Collections.singletonList(targetReplica))))).all().get(); + adminClient.alterPartitionReassignments(Map.of(tp, + Optional.of(new NewPartitionReassignment(List.of(targetReplica))))).all().get(); // let's wait until the LAIR is propagated TestUtils.waitForCondition( - () -> !adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get() + () -> !adminClient.listPartitionReassignments(Set.of(tp)).reassignments().get() .get(tp).addingReplicas().isEmpty(), CLUSTER_WAIT_MS, "Reassignment didn't add the second node" ); @@ -1208,7 +1208,7 @@ public class TopicCommandTest { TestUtils.waitForCondition( () -> { - PartitionReassignment tempReassignments = adminClient.listPartitionReassignments(Collections.singleton(tp)).reassignments().get().get(tp); + PartitionReassignment tempReassignments = adminClient.listPartitionReassignments(Set.of(tp)).reassignments().get().get(tp); reassignmentsRef.set(tempReassignments); return reassignmentsRef.get() != null; }, waitTimeMs, "Reassignments did not become non-null within the specified time" @@ -1216,7 +1216,7 @@ public class TopicCommandTest { assertFalse(reassignmentsRef.get().addingReplicas().isEmpty()); - ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Collections.singleton(tp)); + ToolsTestUtils.removeReplicationThrottleForPartitions(adminClient, brokerIds, Set.of(tp)); TestUtils.waitForCondition( () -> adminClient.listPartitionReassignments().reassignments().get().isEmpty(), CLUSTER_WAIT_MS, String.format("reassignmet not finished after %s ms", CLUSTER_WAIT_MS) @@ -1235,7 +1235,7 @@ public class TopicCommandTest { int partitions = 1; short replicationFactor = 6; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); clusterInstance.waitTopicCreation(testTopicName, partitions); clusterInstance.shutdownBroker(0); @@ -1278,10 +1278,10 @@ public class TopicCommandTest { List newTopics = new ArrayList<>(); Map> fullyReplicatedReplicaAssignmentMap = new HashMap<>(); - fullyReplicatedReplicaAssignmentMap.put(0, Arrays.asList(1, 2, 3)); + fullyReplicatedReplicaAssignmentMap.put(0, List.of(1, 2, 3)); Map> offlineReplicaAssignmentMap = new HashMap<>(); - offlineReplicaAssignmentMap.put(0, Arrays.asList(0)); + offlineReplicaAssignmentMap.put(0, List.of(0)); Map topicConfig = new HashMap<>(); topicConfig.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "6"); @@ -1333,7 +1333,7 @@ public class TopicCommandTest { int partitions = 2; short replicationFactor = 2; - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, partitions, replicationFactor).configs(topicConfig))); clusterInstance.waitTopicCreation(testTopicName, partitions); String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe")); @@ -1345,7 +1345,7 @@ public class TopicCommandTest { public void testDescribeAndListTopicsWithoutInternalTopics(ClusterInstance clusterInstance) throws InterruptedException { String testTopicName = TestUtils.randomString(10); try (Admin adminClient = clusterInstance.admin()) { - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); // test describe @@ -1374,9 +1374,9 @@ public class TopicCommandTest { new ClusterAuthorizationException("Unauthorized")); doReturn(result).when(adminClient).listPartitionReassignments( - Collections.singleton(new TopicPartition(testTopicName, 0)) + Set.of(new TopicPartition(testTopicName, 0)) ); - adminClient.createTopics(Collections.singletonList(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); + adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); String output = captureDescribeTopicStandardOut(clusterInstance, buildTopicCommandOptionsWithBootstrap(clusterInstance, "--describe", "--topic", testTopicName)); @@ -1395,7 +1395,7 @@ public class TopicCommandTest { String topic = "foo_bar"; int partitions = 1; short replicationFactor = 3; - adminClient.createTopics(Collections.singletonList(new NewTopic(topic, partitions, replicationFactor))); + adminClient.createTopics(List.of(new NewTopic(topic, partitions, replicationFactor))); clusterInstance.waitTopicCreation(topic, defaultNumPartitions); assertThrows(TopicExistsException.class, @@ -1444,7 +1444,7 @@ public class TopicCommandTest { List partitionRackMapValueSize = partitionRackMap.values().stream() .map(value -> (int) value.stream().distinct().count()) - .collect(Collectors.toList()); + .toList(); List expected = Collections.nCopies(numPartitions, replicationFactor); assertEquals(expected, partitionRackMapValueSize, "More than one replica of the same partition is assigned to the same rack"); @@ -1502,9 +1502,9 @@ public class TopicCommandTest { String rack; if (brokerRackMapping.containsKey(brokerId)) { rack = brokerRackMapping.get(brokerId); - List partitionRackValues = Stream.of(Collections.singletonList(rack), partitionRackMap.getOrDefault(partitionId, Collections.emptyList())) + List partitionRackValues = Stream.of(List.of(rack), partitionRackMap.getOrDefault(partitionId, List.of())) .flatMap(List::stream) - .collect(Collectors.toList()); + .toList(); partitionRackMap.put(partitionId, partitionRackValues); } else { System.err.printf("No mapping found for %s in `brokerRackMapping`%n", brokerId); @@ -1514,18 +1514,9 @@ public class TopicCommandTest { return new ReplicaDistributions(partitionRackMap, leaderCount, partitionCount); } - private static class ReplicaDistributions { - private final Map> partitionRacks; - private final Map brokerLeaderCount; - private final Map brokerReplicasCount; - - public ReplicaDistributions(Map> partitionRacks, - Map brokerLeaderCount, - Map brokerReplicasCount) { - this.partitionRacks = partitionRacks; - this.brokerLeaderCount = brokerLeaderCount; - this.brokerReplicasCount = brokerReplicasCount; - } + private record ReplicaDistributions(Map> partitionRacks, + Map brokerLeaderCount, + Map brokerReplicasCount) { } private KafkaProducer createProducer(ClusterInstance clusterInstance) { diff --git a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java index c46965a6ee6..17613d98e4b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TransactionsCommandTest.java @@ -57,9 +57,7 @@ import java.io.IOException; import java.io.InputStreamReader; import java.io.PrintStream; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -69,11 +67,6 @@ import java.util.OptionalLong; import java.util.Set; import java.util.concurrent.TimeUnit; -import static java.util.Arrays.asList; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static java.util.Collections.singletonMap; import static org.apache.kafka.common.KafkaFuture.completedFuture; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -161,7 +154,7 @@ public class TransactionsCommandTest { ) throws Exception { DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class); KafkaFuture describeFuture = completedFuture( - new PartitionProducerState(asList( + new PartitionProducerState(List.of( new ProducerState(12345L, 15, 1300, 1599509565L, OptionalInt.of(20), OptionalLong.of(990)), new ProducerState(98765L, 30, 2300, 1599509599L, @@ -170,7 +163,7 @@ public class TransactionsCommandTest { Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture); - Mockito.when(admin.describeProducers(singleton(topicPartition), expectedOptions)).thenReturn(describeResult); + Mockito.when(admin.describeProducers(Set.of(topicPartition), expectedOptions)).thenReturn(describeResult); execute(args); assertNormalExit(); @@ -182,8 +175,8 @@ public class TransactionsCommandTest { assertEquals(expectedHeaders, table.get(0)); Set> expectedRows = Set.of( - asList("12345", "15", "20", "1300", "1599509565", "990"), - asList("98765", "30", "-1", "2300", "1599509599", "None") + List.of("12345", "15", "20", "1300", "1599509565", "990"), + List.of("98765", "30", "-1", "2300", "1599509599", "None") ); assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size()))); } @@ -233,9 +226,9 @@ public class TransactionsCommandTest { assertEquals(expectedHeaders, table.get(0)); Set> expectedRows = Set.of( - asList("foo", "0", "12345", "Ongoing"), - asList("bar", "0", "98765", "PrepareAbort"), - asList("baz", "1", "13579", "CompleteCommit") + List.of("foo", "0", "12345", "Ongoing"), + List.of("bar", "0", "98765", "PrepareAbort"), + List.of("baz", "1", "13579", "CompleteCommit") ); assertEquals(expectedRows, new HashSet<>(table.subList(1, table.size()))); } @@ -302,11 +295,11 @@ public class TransactionsCommandTest { 15, 10000, OptionalLong.of(transactionStartTime), - singleton(new TopicPartition("bar", 0)) + Set.of(new TopicPartition("bar", 0)) )); Mockito.when(describeResult.description(transactionalId)).thenReturn(describeFuture); - Mockito.when(admin.describeTransactions(singleton(transactionalId))).thenReturn(describeResult); + Mockito.when(admin.describeTransactions(Set.of(transactionalId))).thenReturn(describeResult); // Add a little time so that we can see a positive transaction duration in the output time.sleep(5000); @@ -320,7 +313,7 @@ public class TransactionsCommandTest { List expectedHeaders = TransactionsCommand.DescribeTransactionsCommand.HEADERS; assertEquals(expectedHeaders, table.get(0)); - List expectedRow = asList( + List expectedRow = List.of( String.valueOf(coordinatorId), transactionalId, "12345", @@ -463,7 +456,7 @@ public class TransactionsCommandTest { DescribeProducersResult describeResult = Mockito.mock(DescribeProducersResult.class); KafkaFuture describeFuture = completedFuture( - new PartitionProducerState(singletonList( + new PartitionProducerState(List.of( new ProducerState(producerId, producerEpoch, 1300, 1599509565L, OptionalInt.of(coordinatorEpoch), OptionalLong.of(startOffset)) ))); @@ -474,7 +467,7 @@ public class TransactionsCommandTest { topicPartition, producerId, producerEpoch, coordinatorEpoch); Mockito.when(describeResult.partitionResult(topicPartition)).thenReturn(describeFuture); - Mockito.when(admin.describeProducers(singleton(topicPartition))).thenReturn(describeResult); + Mockito.when(admin.describeProducers(Set.of(topicPartition))).thenReturn(describeResult); Mockito.when(abortTransactionResult.all()).thenReturn(abortFuture); Mockito.when(admin.abortTransaction(expectedAbortSpec)).thenReturn(abortTransactionResult); @@ -570,7 +563,7 @@ public class TransactionsCommandTest { OptionalInt coordinatorEpoch, OptionalLong txnStartOffset ) { - PartitionProducerState partitionProducerState = new PartitionProducerState(singletonList( + PartitionProducerState partitionProducerState = new PartitionProducerState(List.of( new ProducerState( producerId, producerEpoch, @@ -583,11 +576,11 @@ public class TransactionsCommandTest { DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class); Mockito.when(result.all()).thenReturn( - completedFuture(singletonMap(topicPartition, partitionProducerState)) + completedFuture(Map.of(topicPartition, partitionProducerState)) ); Mockito.when(admin.describeProducers( - Collections.singletonList(topicPartition), + List.of(topicPartition), new DescribeProducersOptions() )).thenReturn(result); } @@ -596,10 +589,10 @@ public class TransactionsCommandTest { Map descriptions ) { DescribeTransactionsResult result = Mockito.mock(DescribeTransactionsResult.class); - descriptions.forEach((transactionalId, description) -> { + descriptions.forEach((transactionalId, description) -> Mockito.when(result.description(transactionalId)) - .thenReturn(completedFuture(description)); - }); + .thenReturn(completedFuture(description)) + ); Mockito.when(result.all()).thenReturn(completedFuture(descriptions)); Mockito.when(admin.describeTransactions(descriptions.keySet())).thenReturn(result); } @@ -634,7 +627,7 @@ public class TransactionsCommandTest { }; String topic = "foo"; - expectListTopics(singleton(topic)); + expectListTopics(Set.of(topic)); Node node0 = new Node(0, "localhost", 9092); Node node1 = new Node(1, "localhost", 9093); @@ -643,28 +636,28 @@ public class TransactionsCommandTest { TopicPartitionInfo partition0 = new TopicPartitionInfo( 0, node0, - Arrays.asList(node0, node1), - Arrays.asList(node0, node1) + List.of(node0, node1), + List.of(node0, node1) ); TopicPartitionInfo partition1 = new TopicPartitionInfo( 1, node1, - Arrays.asList(node1, node5), - Arrays.asList(node1, node5) + List.of(node1, node5), + List.of(node1, node5) ); TopicDescription description = new TopicDescription( topic, false, - Arrays.asList(partition0, partition1) + List.of(partition0, partition1) ); - expectDescribeTopics(singletonMap(topic, description)); + expectDescribeTopics(Map.of(topic, description)); DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class); - Mockito.when(result.all()).thenReturn(completedFuture(emptyMap())); + Mockito.when(result.all()).thenReturn(completedFuture(Map.of())); Mockito.when(admin.describeProducers( - Collections.singletonList(new TopicPartition(topic, 1)), + List.of(new TopicPartition(topic, 1)), new DescribeProducersOptions().brokerId(brokerId) )).thenReturn(result); @@ -695,28 +688,28 @@ public class TransactionsCommandTest { TopicPartitionInfo partition0 = new TopicPartitionInfo( 0, node0, - Arrays.asList(node0, node1), - Arrays.asList(node0, node1) + List.of(node0, node1), + List.of(node0, node1) ); TopicPartitionInfo partition1 = new TopicPartitionInfo( 1, node1, - Arrays.asList(node1, node5), - Arrays.asList(node1, node5) + List.of(node1, node5), + List.of(node1, node5) ); TopicDescription description = new TopicDescription( topic, false, - Arrays.asList(partition0, partition1) + List.of(partition0, partition1) ); - expectDescribeTopics(singletonMap(topic, description)); + expectDescribeTopics(Map.of(topic, description)); DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class); - Mockito.when(result.all()).thenReturn(completedFuture(emptyMap())); + Mockito.when(result.all()).thenReturn(completedFuture(Map.of())); Mockito.when(admin.describeProducers( - Collections.singletonList(new TopicPartition(topic, 1)), + List.of(new TopicPartition(topic, 1)), new DescribeProducersOptions().brokerId(brokerId) )).thenReturn(result); @@ -744,28 +737,28 @@ public class TransactionsCommandTest { TopicPartitionInfo partition0 = new TopicPartitionInfo( 0, node0, - Arrays.asList(node0, node1), - Arrays.asList(node0, node1) + List.of(node0, node1), + List.of(node0, node1) ); TopicPartitionInfo partition1 = new TopicPartitionInfo( 1, node1, - Arrays.asList(node1, node5), - Arrays.asList(node1, node5) + List.of(node1, node5), + List.of(node1, node5) ); TopicDescription description = new TopicDescription( topic, false, - Arrays.asList(partition0, partition1) + List.of(partition0, partition1) ); - expectDescribeTopics(singletonMap(topic, description)); + expectDescribeTopics(Map.of(topic, description)); DescribeProducersResult result = Mockito.mock(DescribeProducersResult.class); - Mockito.when(result.all()).thenReturn(completedFuture(emptyMap())); + Mockito.when(result.all()).thenReturn(completedFuture(Map.of())); Mockito.when(admin.describeProducers( - Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 1)), + List.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1)), new DescribeProducersOptions() )).thenReturn(result); @@ -851,11 +844,11 @@ public class TransactionsCommandTest { ); expectListTransactions( - new ListTransactionsOptions().filterProducerIds(singleton(producerId)), - singletonMap(1, Collections.emptyList()) + new ListTransactionsOptions().filterProducerIds(Set.of(producerId)), + Map.of(1, List.of()) ); - expectDescribeTransactions(Collections.emptyMap()); + expectDescribeTransactions(Map.of()); execute(args); assertNormalExit(); @@ -907,14 +900,14 @@ public class TransactionsCommandTest { ); expectListTransactions( - new ListTransactionsOptions().filterProducerIds(singleton(producerId)), - singletonMap(1, Collections.singletonList(listing)) + new ListTransactionsOptions().filterProducerIds(Set.of(producerId)), + Map.of(1, List.of(listing)) ); DescribeTransactionsResult result = Mockito.mock(DescribeTransactionsResult.class); Mockito.when(result.description(transactionalId)) .thenReturn(failedFuture(new TransactionalIdNotFoundException(transactionalId + " not found"))); - Mockito.when(admin.describeTransactions(singleton(transactionalId))).thenReturn(result); + Mockito.when(admin.describeTransactions(Set.of(transactionalId))).thenReturn(result); execute(args); assertNormalExit(); @@ -972,8 +965,8 @@ public class TransactionsCommandTest { ); expectListTransactions( - new ListTransactionsOptions().filterProducerIds(singleton(producerId)), - singletonMap(1, Collections.singletonList(listing)) + new ListTransactionsOptions().filterProducerIds(Set.of(producerId)), + Map.of(1, List.of(listing)) ); // Although there is a transaction in progress from the same @@ -986,10 +979,10 @@ public class TransactionsCommandTest { producerEpoch, 60000, OptionalLong.of(time.milliseconds()), - singleton(new TopicPartition("foo", 10)) + Set.of(new TopicPartition("foo", 10)) ); - expectDescribeTransactions(singletonMap(transactionalId, description)); + expectDescribeTransactions(Map.of(transactionalId, description)); execute(args); assertNormalExit(); @@ -1020,7 +1013,7 @@ public class TransactionsCommandTest { long durationMinutes = TimeUnit.MILLISECONDS.toMinutes(time.milliseconds() - lastTimestamp); - List expectedRow = asList( + List expectedRow = List.of( topicPartition.topic(), String.valueOf(topicPartition.partition()), String.valueOf(producerId), @@ -1070,8 +1063,8 @@ public class TransactionsCommandTest { ); expectListTransactions( - new ListTransactionsOptions().filterProducerIds(singleton(producerId)), - singletonMap(1, Collections.singletonList(listing)) + new ListTransactionsOptions().filterProducerIds(Set.of(producerId)), + Map.of(1, List.of(listing)) ); // The coordinator shows an active transaction with the same epoch @@ -1084,10 +1077,10 @@ public class TransactionsCommandTest { producerEpoch, 60000, OptionalLong.of(lastTimestamp), - singleton(topicPartition) + Set.of(topicPartition) ); - expectDescribeTransactions(singletonMap(transactionalId, description)); + expectDescribeTransactions(Map.of(transactionalId, description)); execute(args); assertNormalExit(); @@ -1118,7 +1111,7 @@ public class TransactionsCommandTest { if (line == null) { return null; } else { - return asList(line.split("\\s+")); + return List.of(line.split("\\s+")); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/UserScramCredentialsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/UserScramCredentialsCommandTest.java index 69b33029d9e..65fa28e1c8d 100644 --- a/tools/src/test/java/org/apache/kafka/tools/UserScramCredentialsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/UserScramCredentialsCommandTest.java @@ -25,7 +25,6 @@ import org.apache.kafka.test.NoRetryException; import org.apache.kafka.test.TestUtils; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Objects; import java.util.OptionalInt; @@ -68,8 +67,8 @@ public class UserScramCredentialsCommandTest { throw new RuntimeException(); }); - List commandArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", cluster.bootstrapServers())); - commandArgs.addAll(Arrays.asList(args)); + List commandArgs = new ArrayList<>(List.of("--bootstrap-server", cluster.bootstrapServers())); + commandArgs.addAll(List.of(args)); try { String output = captureStandardOut(() -> ConfigCommand.main(commandArgs.toArray(new String[0]))); return new ConfigCommandResult(output); @@ -162,7 +161,7 @@ public class UserScramCredentialsCommandTest { () -> { try { String output = runConfigCommandViaBroker("--entity-type", "users", "--describe").stdout; - return Arrays.asList(msgs).contains(output); + return List.of(msgs).contains(output); } catch (Exception e) { throw new NoRetryException(e); } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java index 74af4a5c5e2..02ded3d5ca5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerOptionsTest.java @@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -283,7 +282,7 @@ public class ConsoleConsumerOptionsTest { }); // different in all three places - File propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -295,7 +294,7 @@ public class ConsoleConsumerOptionsTest { assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args)); // the same in all three places - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "test-group")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); final String[] args1 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -309,7 +308,7 @@ public class ConsoleConsumerOptionsTest { assertEquals("test-group", props.getProperty("group.id")); // different via --consumer-property and --consumer.config - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args2 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -330,7 +329,7 @@ public class ConsoleConsumerOptionsTest { assertThrows(IllegalArgumentException.class, () -> new ConsoleConsumerOptions(args3)); // different via --group and --consumer.config - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args4 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index eef6ac4cb49..21391230d48 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -61,14 +61,13 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.regex.Pattern; -import static java.util.Collections.singleton; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG; @@ -155,7 +154,7 @@ public class ConsoleConsumerTest { mockConsumer ); - mockConsumer.rebalance(Arrays.asList(tp1, tp2)); + mockConsumer.rebalance(List.of(tp1, tp2)); Map offsets = new HashMap<>(); offsets.put(tp1, startOffset); offsets.put(tp2, startOffset); @@ -234,8 +233,8 @@ public class ConsoleConsumerTest { mockConsumer ); - verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); - verify(mockConsumer).seekToEnd(eq(Collections.singletonList(tp0))); + verify(mockConsumer).assign(eq(List.of(tp0))); + verify(mockConsumer).seekToEnd(eq(List.of(tp0))); consumer.cleanup(); reset(mockConsumer); @@ -249,7 +248,7 @@ public class ConsoleConsumerTest { consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer); - verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); + verify(mockConsumer).assign(eq(List.of(tp0))); verify(mockConsumer).seek(eq(tp0), eq(123L)); consumer.cleanup(); reset(mockConsumer); @@ -264,8 +263,8 @@ public class ConsoleConsumerTest { consumer = new ConsoleConsumer.ConsumerWrapper(new ConsoleConsumerOptions(args), mockConsumer); - verify(mockConsumer).assign(eq(Collections.singletonList(tp0))); - verify(mockConsumer).seekToBeginning(eq(Collections.singletonList(tp0))); + verify(mockConsumer).assign(eq(List.of(tp0))); + verify(mockConsumer).seekToBeginning(eq(List.of(tp0))); consumer.cleanup(); reset(mockConsumer); } @@ -295,7 +294,7 @@ public class ConsoleConsumerTest { try (Admin admin = cluster.admin()) { NewTopic newTopic = new NewTopic(topic, 1, (short) 1); - admin.createTopics(singleton(newTopic)); + admin.createTopics(Set.of(newTopic)); produceMessagesWithTxn(cluster); String[] transactionLogMessageFormatter = createConsoleConsumerArgs(cluster, @@ -334,7 +333,7 @@ public class ConsoleConsumerTest { try (Admin admin = cluster.admin()) { NewTopic newTopic = new NewTopic(topic, 1, (short) 1); - admin.createTopics(singleton(newTopic)); + admin.createTopics(Set.of(newTopic)); produceMessages(cluster); String[] offsetsMessageFormatter = createConsoleConsumerArgs(cluster, @@ -376,7 +375,7 @@ public class ConsoleConsumerTest { try (Admin admin = cluster.admin()) { NewTopic newTopic = new NewTopic(topic, 1, (short) 1); - admin.createTopics(singleton(newTopic)); + admin.createTopics(Set.of(newTopic)); produceMessages(cluster); String[] groupMetadataMessageFormatter = createConsoleConsumerArgs(cluster, diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java index 62b1cc64861..c2b7b73c1a5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleShareConsumerOptionsTest.java @@ -26,7 +26,6 @@ import org.junit.jupiter.api.Test; import java.io.File; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -116,7 +115,7 @@ public class ConsoleShareConsumerOptionsTest { }); // different in all three places - File propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + File propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -128,7 +127,7 @@ public class ConsoleShareConsumerOptionsTest { assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args)); // the same in all three places - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "test-group")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "test-group")); final String[] args1 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -142,7 +141,7 @@ public class ConsoleShareConsumerOptionsTest { assertEquals("test-group", props.getProperty("group.id")); // different via --consumer-property and --consumer-config - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args2 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", @@ -163,7 +162,7 @@ public class ConsoleShareConsumerOptionsTest { assertThrows(IllegalArgumentException.class, () -> new ConsoleShareConsumerOptions(args3)); // different via --group and --consumer-config - propsFile = ToolsTestUtils.tempPropertiesFile(Collections.singletonMap("group.id", "group-from-file")); + propsFile = ToolsTestUtils.tempPropertiesFile(Map.of("group.id", "group-from-file")); final String[] args4 = new String[]{ "--bootstrap-server", "localhost:9092", "--topic", "test", diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/CoordinatorRecordMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/CoordinatorRecordMessageFormatterTest.java index 58cb3f7055e..3b09b26c037 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/CoordinatorRecordMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/CoordinatorRecordMessageFormatterTest.java @@ -28,10 +28,10 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.ByteArrayOutputStream; import java.io.PrintStream; +import java.util.Map; import java.util.Optional; import java.util.stream.Stream; -import static java.util.Collections.emptyMap; import static org.junit.jupiter.api.Assertions.assertEquals; @TestInstance(TestInstance.Lifecycle.PER_CLASS) @@ -59,7 +59,7 @@ public abstract class CoordinatorRecordMessageFormatterTest { ); try (MessageFormatter formatter = formatter()) { - formatter.configure(emptyMap()); + formatter.configure(Map.of()); ByteArrayOutputStream out = new ByteArrayOutputStream(); formatter.writeTo(record, new PrintStream(out)); assertEquals(expectedOutput.replaceAll("\\s+", ""), out.toString()); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java index 0e158bc016b..f3982fa0ffe 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/GroupMetadataMessageFormatterTest.java @@ -24,10 +24,9 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; import org.junit.jupiter.params.provider.Arguments; +import java.util.List; import java.util.stream.Stream; -import static java.util.Collections.singletonList; - public class GroupMetadataMessageFormatterTest extends CoordinatorRecordMessageFormatterTest { private static final OffsetCommitKey OFFSET_COMMIT_KEY = new OffsetCommitKey() @@ -56,7 +55,7 @@ public class GroupMetadataMessageFormatterTest extends CoordinatorRecordMessageF .setGeneration(1) .setProtocol("range") .setLeader("leader") - .setMembers(singletonList(MEMBER_METADATA)) + .setMembers(List.of(MEMBER_METADATA)) .setCurrentStateTimestamp(1234L); @Override diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java index d774d0fed79..ac6dccf1b27 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/OffsetMessageFormatterTest.java @@ -25,7 +25,7 @@ import org.apache.kafka.coordinator.group.generated.OffsetCommitValue; import org.junit.jupiter.params.provider.Arguments; -import java.util.Collections; +import java.util.List; import java.util.stream.Stream; public class OffsetMessageFormatterTest extends CoordinatorRecordMessageFormatterTest { @@ -46,7 +46,7 @@ public class OffsetMessageFormatterTest extends CoordinatorRecordMessageFormatte .setGeneration(1) .setProtocol("range") .setLeader("leader") - .setMembers(Collections.emptyList()); + .setMembers(List.of()); @Override protected CoordinatorRecordMessageFormatter formatter() { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatterTest.java index e2b4ed8aead..7a6f86ace80 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/TransactionLogMessageFormatterTest.java @@ -22,10 +22,9 @@ import org.apache.kafka.coordinator.transaction.generated.TransactionLogValue; import org.junit.jupiter.params.provider.Arguments; +import java.util.List; import java.util.stream.Stream; -import static java.util.Collections.emptyList; - public class TransactionLogMessageFormatterTest extends CoordinatorRecordMessageFormatterTest { private static final TransactionLogKey TXN_LOG_KEY = new TransactionLogKey() @@ -37,7 +36,7 @@ public class TransactionLogMessageFormatterTest extends CoordinatorRecordMessage .setTransactionStartTimestampMs(750L) .setTransactionLastUpdateTimestampMs(1000L) .setTransactionTimeoutMs(500) - .setTransactionPartitions(emptyList()); + .setTransactionPartitions(List.of()); @Override protected CoordinatorRecordMessageFormatter formatter() { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java index a8d47549778..a1afd7658e6 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java @@ -23,7 +23,8 @@ import org.apache.kafka.common.errors.GroupIdNotFoundException; import org.junit.jupiter.api.Test; -import java.util.Collections; +import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutionException; import scala.jdk.javaapi.CollectionConverters; @@ -36,11 +37,11 @@ import static org.junit.jupiter.api.Assertions.fail; public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { @Test public void testDescribeGroupCliWithGroupDescribe() throws Exception { - addAndVerifyAcls(CollectionConverters.asScala(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource()); + addAndVerifyAcls(CollectionConverters.asScala(Set.of(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource()); String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()}; ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs); - try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap())) { + try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = new ConsumerGroupCommand.ConsumerGroupService(opts, Map.of())) { consumerGroupService.describeGroups(); fail("Non-existent group should throw an exception"); } catch (ExecutionException e) { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java index bf67bcf0d87..faf448d1cdb 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupCommandTestUtils.java @@ -24,7 +24,6 @@ import org.apache.kafka.common.utils.Utils; import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; @@ -55,7 +54,7 @@ class ConsumerGroupCommandTestUtils { String topic, Supplier> consumerSupplier) { return buildConsumers(numberOfConsumers, syncCommit, consumerSupplier, - consumer -> consumer.subscribe(Collections.singleton(topic))); + consumer -> consumer.subscribe(Set.of(topic))); } static AutoCloseable buildConsumers( diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java index 242aa1bc894..d5c1b035a44 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ConsumerGroupServiceTest.java @@ -44,9 +44,7 @@ import org.mockito.ArgumentMatcher; import org.mockito.ArgumentMatchers; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -73,11 +71,11 @@ public class ConsumerGroupServiceTest { public static final int NUM_PARTITIONS = 10; - private static final List TOPICS = IntStream.range(0, 5).mapToObj(i -> "testTopic" + i).collect(Collectors.toList()); + private static final List TOPICS = IntStream.range(0, 5).mapToObj(i -> "testTopic" + i).toList(); private static final List TOPIC_PARTITIONS = TOPICS.stream() .flatMap(topic -> IntStream.range(0, NUM_PARTITIONS).mapToObj(i -> new TopicPartition(topic, i))) - .collect(Collectors.toList()); + .toList(); private final Admin admin = mock(Admin.class); @@ -86,7 +84,7 @@ public class ConsumerGroupServiceTest { String[] args = new String[]{"--bootstrap-server", "localhost:9092", "--group", GROUP, "--describe", "--offsets"}; ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args); - when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any())) + when(admin.describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any())) .thenReturn(describeGroupsResult(GroupState.STABLE)); when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any())) .thenReturn(listGroupOffsetsResult(GROUP)); @@ -100,7 +98,7 @@ public class ConsumerGroupServiceTest { assertTrue(statesAndAssignments.getValue().isPresent()); assertEquals(TOPIC_PARTITIONS.size(), statesAndAssignments.getValue().get().size()); - verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); + verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any()); verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any()); verify(admin, times(1)).listOffsets(offsetsArgMatcher(), any()); } @@ -142,7 +140,7 @@ public class ConsumerGroupServiceTest { ConsumerGroupDescription consumerGroupDescription = new ConsumerGroupDescription(GROUP, true, - Collections.singleton( + Set.of( new MemberDescription( "member1", Optional.of("instance1"), "client1", "host1", new MemberAssignment(assignedTopicPartitions), Optional.empty(), Optional.empty(), Optional.empty() @@ -161,12 +159,12 @@ public class ConsumerGroupServiceTest { KafkaFutureImpl future = new KafkaFutureImpl<>(); future.complete(consumerGroupDescription); - when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any())) - .thenReturn(new DescribeConsumerGroupsResult(Collections.singletonMap(GROUP, future))); + when(admin.describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any())) + .thenReturn(new DescribeConsumerGroupsResult(Map.of(GROUP, future))); when(admin.listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any())) .thenReturn( AdminClientTestUtils.listConsumerGroupOffsetsResult( - Collections.singletonMap(GROUP, committedOffsets))); + Map.of(GROUP, committedOffsets))); when(admin.listOffsets( ArgumentMatchers.argThat(offsetsArgMatcher.apply(assignedTopicPartitions)), any() @@ -187,12 +185,12 @@ public class ConsumerGroupServiceTest { results.stream().collect(Collectors.toMap( assignment -> new TopicPartition(assignment.topic.get(), assignment.partition.get()), assignment -> assignment.offset)) - ).orElse(Collections.emptyMap()); + ).orElse(Map.of()); Map> returnedLeaderEpoch = assignments.map(results -> results.stream().collect(Collectors.toMap( assignment -> new TopicPartition(assignment.topic.get(), assignment.partition.get()), assignment -> assignment.leaderEpoch)) - ).orElse(Collections.emptyMap()); + ).orElse(Map.of()); Map> expectedOffsets = Map.of( testTopicPartition0, Optional.empty(), @@ -215,7 +213,7 @@ public class ConsumerGroupServiceTest { assertEquals(expectedOffsets, returnedOffsets); assertEquals(expectedLeaderEpoch, returnedLeaderEpoch); - verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); + verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any()); verify(admin, times(1)).listConsumerGroupOffsets(ArgumentMatchers.eq(listConsumerGroupOffsetsSpec()), any()); verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher.apply(assignedTopicPartitions)), any()); verify(admin, times(1)).listOffsets(ArgumentMatchers.argThat(offsetsArgMatcher.apply(unassignedTopicPartitions)), any()); @@ -223,15 +221,15 @@ public class ConsumerGroupServiceTest { @Test public void testAdminRequestsForResetOffsets() { - List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", GROUP, "--reset-offsets", "--to-latest")); + List args = new ArrayList<>(List.of("--bootstrap-server", "localhost:9092", "--group", GROUP, "--reset-offsets", "--to-latest")); List topicsWithoutPartitionsSpecified = TOPICS.subList(1, TOPICS.size()); - List topicArgs = new ArrayList<>(Arrays.asList("--topic", TOPICS.get(0) + ":" + (IntStream.range(0, NUM_PARTITIONS).mapToObj(Integer::toString).collect(Collectors.joining(","))))); - topicsWithoutPartitionsSpecified.forEach(topic -> topicArgs.addAll(Arrays.asList("--topic", topic))); + List topicArgs = new ArrayList<>(List.of("--topic", TOPICS.get(0) + ":" + (IntStream.range(0, NUM_PARTITIONS).mapToObj(Integer::toString).collect(Collectors.joining(","))))); + topicsWithoutPartitionsSpecified.forEach(topic -> topicArgs.addAll(List.of("--topic", topic))); args.addAll(topicArgs); ConsumerGroupCommand.ConsumerGroupService groupService = consumerGroupService(args.toArray(new String[0])); - when(admin.describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any())) + when(admin.describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any())) .thenReturn(describeGroupsResult(GroupState.DEAD)); when(admin.describeTopics(ArgumentMatchers.eq(topicsWithoutPartitionsSpecified), any())) .thenReturn(describeTopicsResult(topicsWithoutPartitionsSpecified)); @@ -241,16 +239,16 @@ public class ConsumerGroupServiceTest { .thenReturn(listOffsetsResult()); Map> resetResult = groupService.resetOffsets(); - assertEquals(Collections.singleton(GROUP), resetResult.keySet()); + assertEquals(Set.of(GROUP), resetResult.keySet()); assertEquals(new HashSet<>(TOPIC_PARTITIONS), resetResult.get(GROUP).keySet()); - verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(Collections.singletonList(GROUP)), any()); + verify(admin, times(1)).describeConsumerGroups(ArgumentMatchers.eq(List.of(GROUP)), any()); verify(admin, times(1)).describeTopics(ArgumentMatchers.eq(topicsWithoutPartitionsSpecified), any()); verify(admin, times(1)).listOffsets(offsetsArgMatcher(), any()); } private ConsumerGroupCommand.ConsumerGroupService consumerGroupService(String[] args) { - return new ConsumerGroupCommand.ConsumerGroupService(ConsumerGroupCommandOptions.fromArgs(args), Collections.emptyMap()) { + return new ConsumerGroupCommand.ConsumerGroupService(ConsumerGroupCommandOptions.fromArgs(args), Map.of()) { @Override protected Admin createAdminClient(Map configOverrides) { return admin; @@ -265,7 +263,7 @@ public class ConsumerGroupServiceTest { Optional.empty(), Optional.empty(), Optional.empty()); ConsumerGroupDescription description = new ConsumerGroupDescription(GROUP, true, - Collections.singleton(member1), + Set.of(member1), RangeAssignor.class.getName(), GroupType.CLASSIC, groupState, @@ -275,14 +273,14 @@ public class ConsumerGroupServiceTest { Optional.empty()); KafkaFutureImpl future = new KafkaFutureImpl<>(); future.complete(description); - return new DescribeConsumerGroupsResult(Collections.singletonMap(GROUP, future)); + return new DescribeConsumerGroupsResult(Map.of(GROUP, future)); } private ListConsumerGroupOffsetsResult listGroupOffsetsResult(String groupId) { Map offsets = TOPIC_PARTITIONS.stream().collect(Collectors.toMap( Function.identity(), __ -> new OffsetAndMetadata(100))); - return AdminClientTestUtils.listConsumerGroupOffsetsResult(Collections.singletonMap(groupId, offsets)); + return AdminClientTestUtils.listConsumerGroupOffsetsResult(Map.of(groupId, offsets)); } private Map offsetsArgMatcher() { @@ -316,18 +314,14 @@ public class ConsumerGroupServiceTest { } private DescribeTopicsResult describeTopicsResult(Collection topics) { - Map topicDescriptions = new HashMap<>(); - - topics.forEach(topic -> { - List partitions = IntStream.range(0, NUM_PARTITIONS) - .mapToObj(i -> new TopicPartitionInfo(i, Node.noNode(), Collections.emptyList(), Collections.emptyList())) - .collect(Collectors.toList()); - topicDescriptions.put(topic, new TopicDescription(topic, false, partitions)); - }); + var topicDescriptions = topics.stream().collect(Collectors.toMap(Function.identity(), + topic -> new TopicDescription(topic, false, IntStream.range(0, NUM_PARTITIONS) + .mapToObj(i -> new TopicPartitionInfo(i, Node.noNode(), List.of(), List.of())) + .toList()))); return AdminClientTestUtils.describeTopicsResult(topicDescriptions); } private Map listConsumerGroupOffsetsSpec() { - return Collections.singletonMap(GROUP, new ListConsumerGroupOffsetsSpec()); + return Map.of(GROUP, new ListConsumerGroupOffsetsSpec()); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java index 4eda3a1af15..d30c8081440 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java @@ -47,8 +47,6 @@ import java.util.stream.IntStream; import joptsimple.OptionException; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG; @@ -313,7 +311,7 @@ public class DeleteConsumerGroupsTest { cluster, groupId, protocol.name, - emptyMap()); + Map.of()); return ConsumerGroupCommandTestUtils.buildConsumers( 1, @@ -331,7 +329,7 @@ public class DeleteConsumerGroupsTest { ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args); return new ConsumerGroupCommand.ConsumerGroupService( opts, - singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) + Map.of(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java index d273c1fa529..265b880e4ea 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java @@ -41,8 +41,8 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.junit.jupiter.api.Assertions; import java.time.Duration; -import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Map.Entry; @@ -80,7 +80,7 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest { String group = "missing.group"; String topic = "foo:1"; try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = consumerGroupService(getArgs(group, topic))) { - Entry> res = consumerGroupService.deleteOffsets(group, Collections.singletonList(topic)); + Entry> res = consumerGroupService.deleteOffsets(group, List.of(topic)); assertEquals(Errors.GROUP_ID_NOT_FOUND, res.getKey()); } } @@ -185,7 +185,7 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest { private static ConsumerGroupCommand.ConsumerGroupService consumerGroupService(String[] args) { return new ConsumerGroupCommand.ConsumerGroupService( ConsumerGroupCommandOptions.fromArgs(args), - Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) + Map.of(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); } @@ -197,7 +197,7 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest { return () -> { String topic = inputPartition >= 0 ? inputTopic + ":" + inputPartition : inputTopic; try (ConsumerGroupCommand.ConsumerGroupService consumerGroupService = consumerGroupService(getArgs(inputGroup, topic))) { - Entry> res = consumerGroupService.deleteOffsets(inputGroup, Collections.singletonList(topic)); + Entry> res = consumerGroupService.deleteOffsets(inputGroup, List.of(topic)); Errors topLevelError = res.getKey(); Map partitions = res.getValue(); TopicPartition tp = new TopicPartition(inputTopic, expectedPartition); @@ -219,7 +219,7 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest { Runnable validateRunnable) { produceRecord(inputTopic); try (Consumer consumer = createConsumer(inputGroup, groupProtocol)) { - consumer.subscribe(Collections.singletonList(inputTopic)); + consumer.subscribe(List.of(inputTopic)); ConsumerRecords records = consumer.poll(Duration.ofMillis(DEFAULT_MAX_WAIT_MS)); Assertions.assertNotEquals(0, records.count()); consumer.commitSync(); @@ -260,14 +260,14 @@ public class DeleteOffsetsConsumerGroupCommandIntegrationTest { } private void createTopic(String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.createTopics(List.of(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); } } private void removeTopic(String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.deleteTopics(Collections.singletonList(topic)).all()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.deleteTopics(List.of(topic)).all()); } } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java index a3f1009e884..b15b4fe9d45 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java @@ -52,7 +52,6 @@ import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -65,7 +64,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; -import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -106,7 +104,7 @@ public class DescribeConsumerGroupTest { for (List describeType : DESCRIBE_TYPES) { // note the group to be queried is a different (non-existing) group - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup)); cgcArgs.addAll(describeType); try (ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0]))) { service.describeGroups(); @@ -130,7 +128,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // note the group to be queried is a different (non-existing) group ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup}) ) { @@ -151,7 +149,7 @@ public class DescribeConsumerGroupTest { String topic = TOPIC_PREFIX + groupProtocol.name(); String group = GROUP_PREFIX + groupProtocol.name(); createTopic(topic); - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // note the group to be queried is a different (non-existing) group ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup}) ) { @@ -172,7 +170,7 @@ public class DescribeConsumerGroupTest { String topic = TOPIC_PREFIX + groupProtocol.name(); String group = GROUP_PREFIX + groupProtocol.name(); createTopic(topic); - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // note the group to be queried is a different (non-existing) group ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", missingGroup}) ) { @@ -295,7 +293,7 @@ public class DescribeConsumerGroupTest { ); try (AutoCloseable consumerConsumer = consumerGroupClosable(GroupProtocol.CONSUMER, group, topic, Map.of(ConsumerConfig.CLIENT_ID_CONFIG, consumerClientId)); - ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])); + ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])) ) { TestUtils.waitForCondition(() -> { Entry res = ToolsTestUtils.grabConsoleOutputAndError(describeGroups(service)); @@ -378,14 +376,14 @@ public class DescribeConsumerGroupTest { List groups = new ArrayList<>(); for (List describeType : DESCRIBE_TYPES) { String group = GROUP_PREFIX + groupProtocol.name() + "." + String.join("", describeType); - groups.addAll(Arrays.asList("--group", group)); - protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap())); + groups.addAll(List.of("--group", group)); + protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, topic, Map.of())); } int expectedNumLines = DESCRIBE_TYPES.size() * 2; for (List describeType : DESCRIBE_TYPES) { - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe")); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe")); cgcArgs.addAll(groups); cgcArgs.addAll(describeType); try (ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0]))) { @@ -420,7 +418,7 @@ public class DescribeConsumerGroupTest { for (List describeType : DESCRIBE_TYPES) { String group = GROUP_PREFIX + groupProtocol.name() + "." + String.join("", describeType); groups.add(group); - protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap())); + protocolConsumerGroupExecutors.add(consumerGroupClosable(groupProtocol, group, topic, Map.of())); } int expectedNumLines = DESCRIBE_TYPES.size() * 2; for (List describeType : DESCRIBE_TYPES) { @@ -456,7 +454,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -496,12 +494,12 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}); - Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers())) + Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers())) ) { TestUtils.waitForCondition(() -> { - ConsumerGroupDescription consumerGroupDescription = admin.describeConsumerGroups(Collections.singleton(group)).describedGroups().get(group).get(); + ConsumerGroupDescription consumerGroupDescription = admin.describeConsumerGroups(Set.of(group)).describedGroups().get(group).get(); return consumerGroupDescription.members().size() == 1 && consumerGroupDescription.members().iterator().next().assignment().topicPartitions().size() == 1; }, "Expected a 'Stable' group status, rows and valid member information for group " + group + "."); @@ -523,7 +521,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.singletonMap(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, groupProtocol == GroupProtocol.CONSUMER ? "range" : "")); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, groupProtocol == GroupProtocol.CONSUMER ? "range" : "")); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -550,10 +548,10 @@ public class DescribeConsumerGroupTest { try { String expectedName; if (groupProtocol.equals(GroupProtocol.CONSUMER)) { - protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CONSUMER, group, topic, Collections.singletonMap(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range")); + protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CONSUMER, group, topic, Map.of(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, "range")); expectedName = RangeAssignor.RANGE_ASSIGNOR_NAME; } else { - protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, group, topic, Collections.singletonMap(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RoundRobinAssignor.class.getName())); + protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, group, topic, Map.of(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, RoundRobinAssignor.class.getName())); expectedName = RoundRobinAssignor.ROUNDROBIN_ASSIGNOR_NAME; } @@ -587,7 +585,7 @@ public class DescribeConsumerGroupTest { List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group)); cgcArgs.addAll(describeType); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])) ) { TestUtils.waitForCondition(() -> { @@ -615,7 +613,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -631,7 +629,7 @@ public class DescribeConsumerGroupTest { Entry, Optional>> offsets = service.collectGroupOffsets(group); Optional state = offsets.getKey(); Optional> assignments = offsets.getValue(); - List testGroupAssignments = assignments.get().stream().filter(a -> Objects.equals(a.group, group)).collect(Collectors.toList()); + List testGroupAssignments = assignments.get().stream().filter(a -> Objects.equals(a.group, group)).toList(); PartitionAssignmentState assignment = testGroupAssignments.get(0); return state.map(s -> s.equals(GroupState.EMPTY)).orElse(false) && testGroupAssignments.size() == 1 && @@ -652,7 +650,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -681,7 +679,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -712,10 +710,10 @@ public class DescribeConsumerGroupTest { for (List describeType : DESCRIBE_TYPES) { String group = GROUP_PREFIX + groupProtocol.name() + String.join("", describeType); - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group)); cgcArgs.addAll(describeType); // run two consumers in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])) ) { TestUtils.waitForCondition(() -> { @@ -739,7 +737,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run two consumers in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -762,7 +760,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run two consumers in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -792,7 +790,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run two consumers in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -812,10 +810,10 @@ public class DescribeConsumerGroupTest { for (List describeType : DESCRIBE_TYPES) { String group = GROUP_PREFIX + groupProtocol.name() + String.join("", describeType); - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group)); cgcArgs.addAll(describeType); // run two consumers in the group consuming from a two-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])) ) { TestUtils.waitForCondition(() -> { @@ -839,7 +837,7 @@ public class DescribeConsumerGroupTest { createTopic(topic, 2); // run two consumers in the group consuming from a two-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -863,7 +861,7 @@ public class DescribeConsumerGroupTest { createTopic(topic, 2); // run two consumers in the group consuming from a two-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -891,7 +889,7 @@ public class DescribeConsumerGroupTest { createTopic(topic, 2); // run two consumers in the group consuming from a two-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap(), 2); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(), 2); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -911,7 +909,7 @@ public class DescribeConsumerGroupTest { String group = GROUP_PREFIX + groupProtocol.name(); createTopic(topic, 2); - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, group, Set.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1)), Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, group, Set.of(new TopicPartition(topic, 0), new TopicPartition(topic, 1)), Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -936,11 +934,11 @@ public class DescribeConsumerGroupTest { String group = GROUP_PREFIX + groupProtocol.name() + String.join("", describeType); // set the group initialization timeout too low for the group to stabilize - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--timeout", "1", "--group", group)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--timeout", "1", "--group", group)); cgcArgs.addAll(describeType); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(cgcArgs.toArray(new String[0])) ) { ExecutionException e = assertThrows(ExecutionException.class, service::describeGroups); @@ -961,7 +959,7 @@ public class DescribeConsumerGroupTest { // complete before the timeout expires // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // set the group initialization timeout too low for the group to stabilize ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group, "--timeout", "1"}) ) { @@ -983,7 +981,7 @@ public class DescribeConsumerGroupTest { // complete before the timeout expires // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // set the group initialization timeout too low for the group to stabilize ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group, "--timeout", "1"}) ) { @@ -1007,7 +1005,7 @@ public class DescribeConsumerGroupTest { // complete before the timeout expires // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.emptyMap()); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of()); // set the group initialization timeout too low for the group to stabilize ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group, "--timeout", "1"}) ) { @@ -1026,7 +1024,7 @@ public class DescribeConsumerGroupTest { createTopic(topic); // run one consumer in the group consuming from a single-partition topic - try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Collections.singletonMap(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")); + try (AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, group, topic, Map.of(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")); ConsumerGroupCommand.ConsumerGroupService service = consumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--describe", "--group", group}) ) { TestUtils.waitForCondition(() -> { @@ -1175,7 +1173,7 @@ public class DescribeConsumerGroupTest { private static ConsumerGroupCommand.ConsumerGroupService consumerGroupService(String[] args) { return new ConsumerGroupCommand.ConsumerGroupService( ConsumerGroupCommandOptions.fromArgs(args), - Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) + Map.of(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); } @@ -1184,20 +1182,20 @@ public class DescribeConsumerGroupTest { } private void createTopic(String topic, int numPartitions) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, numPartitions, (short) 1))).topicId(topic).get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.createTopics(List.of(new NewTopic(topic, numPartitions, (short) 1))).topicId(topic).get()); } } private void deleteConsumerGroups(Collection groupIds) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { Assertions.assertDoesNotThrow(() -> admin.deleteConsumerGroups(groupIds).all().get()); } } private void deleteTopic(String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.deleteTopics(Collections.singletonList(topic)).topicNameValues().get(topic).get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.deleteTopics(List.of(topic)).topicNameValues().get(topic).get()); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index fd50aa3b7d7..8775d26869e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -43,7 +43,6 @@ import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -53,11 +52,9 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; import joptsimple.OptionException; -import static java.util.Collections.emptyMap; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_PROTOCOL_CONFIG; @@ -105,12 +102,12 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + i; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable topicConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, topicGroup, topic); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list"}) ) { - Set expectedGroups = set(Arrays.asList(topicPartitionsGroup, topicGroup, protocolGroup)); + Set expectedGroups = set(List.of(topicPartitionsGroup, topicGroup, protocolGroup)); final AtomicReference foundGroups = new AtomicReference<>(); TestUtils.waitForCondition(() -> { @@ -119,7 +116,7 @@ public class ListConsumerGroupTest { }, "Expected --list to show groups " + expectedGroups + ", but found " + foundGroups.get() + "."); } - removeConsumer(set(Arrays.asList(topicPartitionsGroup, topicGroup, protocolGroup))); + removeConsumer(set(List.of(topicPartitionsGroup, topicGroup, protocolGroup))); deleteTopic(topic); } } @@ -139,7 +136,7 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + i; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"}) ) { @@ -160,7 +157,7 @@ public class ListConsumerGroupTest { assertGroupListing( service, - Collections.emptySet(), + Set.of(), EnumSet.allOf(GroupState.class), expectedListing ); @@ -176,20 +173,20 @@ public class ListConsumerGroupTest { assertGroupListing( service, - Collections.emptySet(), + Set.of(), Set.of(GroupState.STABLE), expectedListing ); assertGroupListing( service, - Collections.emptySet(), + Set.of(), Set.of(GroupState.PREPARING_REBALANCE), - Collections.emptySet() + Set.of() ); } - removeConsumer(set(Arrays.asList(topicPartitionsGroup, protocolGroup))); + removeConsumer(set(List.of(topicPartitionsGroup, protocolGroup))); deleteTopic(topic); } } @@ -202,7 +199,7 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"}) ) { @@ -224,8 +221,8 @@ public class ListConsumerGroupTest { // No filters explicitly mentioned. Expectation is that all groups are returned. assertGroupListing( service, - Collections.emptySet(), - Collections.emptySet(), + Set.of(), + Set.of(), expectedListing ); @@ -235,14 +232,14 @@ public class ListConsumerGroupTest { assertGroupListing( service, Set.of(GroupType.CONSUMER), - Collections.emptySet(), - Collections.emptySet() + Set.of(), + Set.of() ); assertGroupListing( service, Set.of(GroupType.CLASSIC), - Collections.emptySet(), + Set.of(), expectedListing ); } @@ -257,7 +254,7 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable topicConsumerGroupExecutor = consumerGroupClosable(GroupProtocol.CLASSIC, topicGroup, topic); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(new String[]{"--bootstrap-server", clusterInstance.bootstrapServers(), "--list"}) @@ -288,8 +285,8 @@ public class ListConsumerGroupTest { assertGroupListing( service, - Collections.emptySet(), - Collections.emptySet(), + Set.of(), + Set.of(), expectedListing ); @@ -307,7 +304,7 @@ public class ListConsumerGroupTest { assertGroupListing( service, Set.of(GroupType.CONSUMER), - Collections.emptySet(), + Set.of(), expectedListing ); @@ -329,7 +326,7 @@ public class ListConsumerGroupTest { assertGroupListing( service, Set.of(GroupType.CLASSIC), - Collections.emptySet(), + Set.of(), expectedListing ); } @@ -343,78 +340,78 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic) ) { validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), - Collections.emptyList(), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), + List.of(), Set.of( - Collections.singletonList(protocolGroup), - Collections.singletonList(topicPartitionsGroup) + List.of(protocolGroup), + List.of(topicPartitionsGroup) ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), - Arrays.asList("GROUP", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), + List.of("GROUP", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Stable"), - Arrays.asList(topicPartitionsGroup, "Empty") + List.of(protocolGroup, "Stable"), + List.of(topicPartitionsGroup, "Empty") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), - Arrays.asList("GROUP", "TYPE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), + List.of("GROUP", "TYPE"), Set.of( - Arrays.asList(protocolGroup, "Classic"), - Arrays.asList(topicPartitionsGroup, "Classic") + List.of(protocolGroup, "Classic"), + List.of(topicPartitionsGroup, "Classic") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), - Arrays.asList("GROUP", "TYPE", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), + List.of("GROUP", "TYPE", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Classic", "Stable"), - Arrays.asList(topicPartitionsGroup, "Classic", "Empty") + List.of(protocolGroup, "Classic", "Stable"), + List.of(topicPartitionsGroup, "Classic", "Empty") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "Stable"), - Arrays.asList("GROUP", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "Stable"), + List.of("GROUP", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Stable") + List.of(protocolGroup, "Stable") ) ); // Check case-insensitivity in state filter. validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "stable"), - Arrays.asList("GROUP", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state", "stable"), + List.of("GROUP", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Stable") + List.of(protocolGroup, "Stable") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "Classic"), - Arrays.asList("GROUP", "TYPE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "Classic"), + List.of("GROUP", "TYPE"), Set.of( - Arrays.asList(protocolGroup, "Classic"), - Arrays.asList(topicPartitionsGroup, "Classic") + List.of(protocolGroup, "Classic"), + List.of(topicPartitionsGroup, "Classic") ) ); // Check case-insensitivity in type filter. validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "classic"), - Arrays.asList("GROUP", "TYPE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "classic"), + List.of("GROUP", "TYPE"), Set.of( - Arrays.asList(protocolGroup, "Classic"), - Arrays.asList(topicPartitionsGroup, "Classic") + List.of(protocolGroup, "Classic"), + List.of(topicPartitionsGroup, "Classic") ) ); } @@ -428,58 +425,58 @@ public class ListConsumerGroupTest { String topicPartitionsGroup = TOPIC_PARTITIONS_GROUP_PREFIX + "0"; createTopic(topic); - try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Collections.singleton(new TopicPartition(topic, 0))); + try (AutoCloseable topicPartitionsConsumerGroupExecutor = consumerGroupClosable(topicPartitionsGroup, Set.of(new TopicPartition(topic, 0))); AutoCloseable protocolConsumerGroupExecutor = consumerGroupClosable(groupProtocol, protocolGroup, topic) ) { validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), - Collections.emptyList(), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list"), + List.of(), Set.of( - Collections.singletonList(protocolGroup), - Collections.singletonList(topicPartitionsGroup) + List.of(protocolGroup), + List.of(topicPartitionsGroup) ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), - Arrays.asList("GROUP", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--state"), + List.of("GROUP", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Stable"), - Arrays.asList(topicPartitionsGroup, "Empty") + List.of(protocolGroup, "Stable"), + List.of(topicPartitionsGroup, "Empty") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), - Arrays.asList("GROUP", "TYPE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type"), + List.of("GROUP", "TYPE"), Set.of( - Arrays.asList(protocolGroup, "Consumer"), - Arrays.asList(topicPartitionsGroup, "Classic") + List.of(protocolGroup, "Consumer"), + List.of(topicPartitionsGroup, "Classic") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), - Arrays.asList("GROUP", "TYPE", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "--state"), + List.of("GROUP", "TYPE", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Consumer", "Stable"), - Arrays.asList(topicPartitionsGroup, "Classic", "Empty") + List.of(protocolGroup, "Consumer", "Stable"), + List.of(topicPartitionsGroup, "Classic", "Empty") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer"), - Arrays.asList("GROUP", "TYPE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer"), + List.of("GROUP", "TYPE"), Set.of( - Arrays.asList(protocolGroup, "Consumer") + List.of(protocolGroup, "Consumer") ) ); validateListOutput( - Arrays.asList("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer", "--state", "Stable"), - Arrays.asList("GROUP", "TYPE", "STATE"), + List.of("--bootstrap-server", clusterInstance.bootstrapServers(), "--list", "--type", "consumer", "--state", "Stable"), + List.of("GROUP", "TYPE", "STATE"), Set.of( - Arrays.asList(protocolGroup, "Consumer", "Stable") + List.of(protocolGroup, "Consumer", "Stable") ) ); } @@ -489,7 +486,7 @@ public class ListConsumerGroupTest { Map configs = composeConfigs( groupId, protocol.name, - emptyMap() + Map.of() ); return ConsumerGroupCommandTestUtils.buildConsumers( @@ -504,7 +501,7 @@ public class ListConsumerGroupTest { Map configs = composeConfigs( groupId, GroupProtocol.CLASSIC.name, - emptyMap() + Map.of() ); return ConsumerGroupCommandTestUtils.buildConsumers( @@ -533,26 +530,26 @@ public class ListConsumerGroupTest { ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(args); ConsumerGroupCommand.ConsumerGroupService service = new ConsumerGroupCommand.ConsumerGroupService( opts, - Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) + Map.of(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); return service; } private void createTopic(String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.createTopics(Collections.singletonList(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.createTopics(List.of(new NewTopic(topic, 1, (short) 1))).topicId(topic).get()); } } private void deleteTopic(String topic) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - Assertions.assertDoesNotThrow(() -> admin.deleteTopics(Collections.singleton(topic)).all().get()); + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + Assertions.assertDoesNotThrow(() -> admin.deleteTopics(Set.of(topic)).all().get()); } } private void removeConsumer(Set groupIds) { - try (Admin admin = Admin.create(Collections.singletonMap(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { Assertions.assertDoesNotThrow(() -> admin.deleteConsumerGroups(groupIds).all().get()); } } @@ -603,7 +600,7 @@ public class ListConsumerGroupTest { // Parse the header if one is expected. if (!expectedHeader.isEmpty()) { if (lines.length == 0) return false; - List header = Arrays.stream(lines[index++].split("\\s+")).collect(Collectors.toList()); + List header = Arrays.stream(lines[index++].split("\\s+")).toList(); if (!expectedHeader.equals(header)) { return false; } @@ -612,7 +609,7 @@ public class ListConsumerGroupTest { // Parse the groups. Set> groups = new HashSet<>(); for (; index < lines.length; index++) { - groups.add(Arrays.stream(lines[index].split("\\s+")).collect(Collectors.toList())); + groups.add(Arrays.stream(lines[index].split("\\s+")).toList()); } return expectedRows.equals(groups); }, () -> String.format("Expected header=%s and groups=%s, but found:%n%s", expectedHeader, expectedRows, out.get())); @@ -627,22 +624,22 @@ class ListConsumerGroupUnitTest { @Test public void testConsumerGroupStatesFromString() { Set result = ConsumerGroupCommand.groupStatesFromString("Stable"); - Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singleton(GroupState.STABLE)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(Set.of(GroupState.STABLE)), result); result = ConsumerGroupCommand.groupStatesFromString("Stable, PreparingRebalance"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupState.STABLE, GroupState.PREPARING_REBALANCE)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupState.STABLE, GroupState.PREPARING_REBALANCE)), result); result = ConsumerGroupCommand.groupStatesFromString("Dead,CompletingRebalance,"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupState.DEAD, GroupState.COMPLETING_REBALANCE)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupState.DEAD, GroupState.COMPLETING_REBALANCE)), result); result = ConsumerGroupCommand.groupStatesFromString("stable"); - Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singletonList(GroupState.STABLE)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupState.STABLE)), result); result = ConsumerGroupCommand.groupStatesFromString("stable, assigning"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupState.STABLE, GroupState.ASSIGNING)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupState.STABLE, GroupState.ASSIGNING)), result); result = ConsumerGroupCommand.groupStatesFromString("dead,reconciling,"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupState.DEAD, GroupState.RECONCILING)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupState.DEAD, GroupState.RECONCILING)), result); Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.groupStatesFromString("bad, wrong")); @@ -654,13 +651,13 @@ class ListConsumerGroupUnitTest { @Test public void testConsumerGroupTypesFromString() { Set result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer"); - Assertions.assertEquals(ListConsumerGroupTest.set(Collections.singleton(GroupType.CONSUMER)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(Set.of(GroupType.CONSUMER)), result); result = ConsumerGroupCommand.consumerGroupTypesFromString("consumer, classic"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupType.CONSUMER, GroupType.CLASSIC)), result); result = ConsumerGroupCommand.consumerGroupTypesFromString("Consumer, Classic"); - Assertions.assertEquals(ListConsumerGroupTest.set(Arrays.asList(GroupType.CONSUMER, GroupType.CLASSIC)), result); + Assertions.assertEquals(ListConsumerGroupTest.set(List.of(GroupType.CONSUMER, GroupType.CLASSIC)), result); Assertions.assertThrows(IllegalArgumentException.class, () -> ConsumerGroupCommand.consumerGroupTypesFromString("Share")); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java index 5bf9da0c370..1c9ab9cf98b 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -56,16 +56,11 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.function.Function; import java.util.function.Supplier; -import java.util.stream.Collectors; import java.util.stream.IntStream; import joptsimple.OptionException; import static java.time.LocalDateTime.now; -import static java.util.Arrays.asList; -import static java.util.Collections.singleton; -import static java.util.Collections.singletonList; -import static java.util.Collections.singletonMap; import static java.util.stream.Collectors.toMap; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.admin.AdminClientConfig.RETRIES_CONFIG; @@ -121,23 +116,23 @@ public class ResetConsumerGroupOffsetTest { } private String[] buildArgsForGroups(ClusterInstance cluster, List groups, String... args) { - List res = new ArrayList<>(asList(basicArgs(cluster))); + List res = new ArrayList<>(List.of(basicArgs(cluster))); for (String group : groups) { res.add("--group"); res.add(group); } - res.addAll(asList(args)); + res.addAll(List.of(args)); return res.toArray(new String[0]); } private String[] buildArgsForGroup(ClusterInstance cluster, String group, String... args) { - return buildArgsForGroups(cluster, singletonList(group), args); + return buildArgsForGroups(cluster, List.of(group), args); } private String[] buildArgsForAllGroups(ClusterInstance cluster, String... args) { - List res = new ArrayList<>(asList(basicArgs(cluster))); + List res = new ArrayList<>(List.of(basicArgs(cluster))); res.add("--all-groups"); - res.addAll(asList(args)); + res.addAll(List.of(args)); return res.toArray(new String[0]); } @@ -183,11 +178,11 @@ public class ResetConsumerGroupOffsetTest { String[] args = buildArgsForGroup(cluster, group, "--topic", topic, "--to-offset", "50"); produceMessages(cluster, topic, 100); - resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, args, 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), - 50, true, singletonList(topic)); + 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--execute"), - 50, false, singletonList(topic)); + 50, false, List.of(topic)); } @ClusterTest @@ -205,11 +200,11 @@ public class ResetConsumerGroupOffsetTest { } String[] args = buildArgsForGroups(cluster, groups, "--topic", topic, "--to-offset", "50"); - resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, args, 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), - 50, true, singletonList(topic)); + 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--execute"), - 50, false, singletonList(topic)); + 50, false, List.of(topic)); } } @@ -227,11 +222,11 @@ public class ResetConsumerGroupOffsetTest { awaitConsumerProgress(cluster, topic, group, 100L); } } - resetAndAssertOffsets(cluster, args, 50, true, singletonList(topic)); + resetAndAssertOffsets(cluster, args, 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--dry-run"), - 50, true, singletonList(topic)); + 50, true, List.of(topic)); resetAndAssertOffsets(cluster, addTo(args, "--execute"), - 50, false, singletonList(topic)); + 50, false, List.of(topic)); } } @@ -346,9 +341,9 @@ public class ResetConsumerGroupOffsetTest { String[] args = buildArgsForGroup(cluster, group, "--topic", topic, "--by-duration", "PT1M", "--execute"); try (Admin admin = cluster.admin()) { - admin.createTopics(singleton(new NewTopic(topic, 1, (short) 1))).all().get(); - resetAndAssertOffsets(cluster, args, 0, false, singletonList(topic)); - admin.deleteTopics(singleton(topic)).all().get(); + admin.createTopics(Set.of(new NewTopic(topic, 1, (short) 1))).all().get(); + resetAndAssertOffsets(cluster, args, 0, false, List.of(topic)); + admin.deleteTopics(Set.of(topic)).all().get(); } } @@ -476,7 +471,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { - admin.createTopics(singleton(new NewTopic(topic, 2, (short) 1))).all().get(); + admin.createTopics(Set.of(new NewTopic(topic, 2, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic, group, 2, groupProtocol); Map priorCommittedOffsets = committedOffsets(cluster, topic, group); @@ -487,7 +482,7 @@ public class ResetConsumerGroupOffsetTest { expectedOffsets.put(tp1, 0L); resetAndAssertOffsetsCommitted(cluster, service, expectedOffsets, topic); - admin.deleteTopics(singleton(topic)).all().get(); + admin.deleteTopics(Set.of(topic)).all().get(); } } } @@ -505,7 +500,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { - admin.createTopics(asList(new NewTopic(topic1, 1, (short) 1), + admin.createTopics(List.of(new NewTopic(topic1, 1, (short) 1), new NewTopic(topic2, 1, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic1, group, 1, groupProtocol); @@ -519,10 +514,10 @@ public class ResetConsumerGroupOffsetTest { expMap.put(tp1, 0L); expMap.put(tp2, 0L); assertEquals(expMap, allResetOffsets); - assertEquals(singletonMap(tp1, 0L), committedOffsets(cluster, topic1, group)); - assertEquals(singletonMap(tp2, 0L), committedOffsets(cluster, topic2, group)); + assertEquals(Map.of(tp1, 0L), committedOffsets(cluster, topic1, group)); + assertEquals(Map.of(tp2, 0L), committedOffsets(cluster, topic2, group)); - admin.deleteTopics(asList(topic1, topic2)).all().get(); + admin.deleteTopics(List.of(topic1, topic2)).all().get(); } } } @@ -540,7 +535,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { - admin.createTopics(asList(new NewTopic(topic1, 2, (short) 1), + admin.createTopics(List.of(new NewTopic(topic1, 2, (short) 1), new NewTopic(topic2, 2, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic1, group, 2, groupProtocol); @@ -563,7 +558,7 @@ public class ResetConsumerGroupOffsetTest { priorCommittedOffsets2.put(tp2, 0L); assertEquals(priorCommittedOffsets2, committedOffsets(cluster, topic2, group)); - admin.deleteTopics(asList(topic1, topic2)).all().get(); + admin.deleteTopics(List.of(topic1, topic2)).all().get(); } } } @@ -584,7 +579,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs)) { - admin.createTopics(singleton(new NewTopic(topic, 2, (short) 1))).all().get(); + admin.createTopics(Set.of(new NewTopic(topic, 2, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic, group, 2, groupProtocol); Map> exportedOffsets = service.resetOffsets(); @@ -603,7 +598,7 @@ public class ResetConsumerGroupOffsetTest { assertEquals(exp1, toOffsetMap(importedOffsets.get(group))); } - admin.deleteTopics(singleton(topic)); + admin.deleteTopics(Set.of(topic)); } } } @@ -622,14 +617,14 @@ public class ResetConsumerGroupOffsetTest { TopicPartition t1p1 = new TopicPartition(topic1, 1); TopicPartition t2p0 = new TopicPartition(topic2, 0); TopicPartition t2p1 = new TopicPartition(topic2, 1); - String[] cgcArgs = buildArgsForGroups(cluster, asList(group1, group2), + String[] cgcArgs = buildArgsForGroups(cluster, List.of(group1, group2), "--all-topics", "--to-offset", "2", "--export"); File file = TestUtils.tempFile("reset", ".csv"); try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(cgcArgs)) { - admin.createTopics(asList(new NewTopic(topic1, 2, (short) 1), + admin.createTopics(List.of(new NewTopic(topic1, 2, (short) 1), new NewTopic(topic2, 2, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic1, group1, 1, groupProtocol); @@ -653,7 +648,7 @@ public class ResetConsumerGroupOffsetTest { assertEquals(exp2, toOffsetMap(exportedOffsets.get(group2))); // Multiple --group's offset import - String[] cgcArgsExec = buildArgsForGroups(cluster, asList(group1, group2), + String[] cgcArgsExec = buildArgsForGroups(cluster, List.of(group1, group2), "--all-topics", "--from-file", file.getCanonicalPath(), "--dry-run"); try (ConsumerGroupCommand.ConsumerGroupService serviceExec = getConsumerGroupService(cgcArgsExec)) { @@ -670,7 +665,7 @@ public class ResetConsumerGroupOffsetTest { assertEquals(exp1, toOffsetMap(importedOffsets2.get(group1))); } - admin.deleteTopics(asList(topic1, topic2)); + admin.deleteTopics(List.of(topic1, topic2)); } } } @@ -695,7 +690,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { - admin.createTopics(singleton(new NewTopic(topic, 3, (short) 1))).all().get(); + admin.createTopics(Set.of(new NewTopic(topic, 3, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic, group, 2, GroupProtocol.CLASSIC); assertDoesNotThrow(() -> resetOffsets(service)); // shutdown a broker to make some partitions missing leader @@ -714,7 +709,7 @@ public class ResetConsumerGroupOffsetTest { try (Admin admin = cluster.admin(); ConsumerGroupCommand.ConsumerGroupService service = getConsumerGroupService(args)) { - admin.createTopics(singleton(new NewTopic(topic, 1, (short) 1))).all().get(); + admin.createTopics(Set.of(new NewTopic(topic, 1, (short) 1))).all().get(); produceConsumeAndShutdown(cluster, topic, group, 2, GroupProtocol.CLASSIC); assertThrows(UnknownTopicOrPartitionException.class, () -> resetOffsets(service)); } @@ -731,7 +726,7 @@ public class ResetConsumerGroupOffsetTest { private Map committedOffsets(ClusterInstance cluster, String topic, String group) { - try (Admin admin = Admin.create(singletonMap(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { return admin.listConsumerGroupOffsets(group) .all().get() .get(group).entrySet() @@ -746,13 +741,13 @@ public class ResetConsumerGroupOffsetTest { private ConsumerGroupCommand.ConsumerGroupService getConsumerGroupService(String[] args) { return new ConsumerGroupCommand.ConsumerGroupService( ConsumerGroupCommandOptions.fromArgs(args), - singletonMap(RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE))); + Map.of(RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE))); } private void produceMessages(ClusterInstance cluster, String topic, int numMessages) { List> records = IntStream.range(0, numMessages) .mapToObj(i -> new ProducerRecord(topic, new byte[100 * 1000])) - .collect(Collectors.toList()); + .toList(); produceMessages(cluster, records); } @@ -775,7 +770,7 @@ public class ResetConsumerGroupOffsetTest { String topic, String[] args, long expectedOffset) { - resetAndAssertOffsets(cluster, args, expectedOffset, false, singletonList(topic)); + resetAndAssertOffsets(cluster, args, expectedOffset, false, List.of(topic)); } private void resetAndAssertOffsets(ClusterInstance cluster, @@ -802,7 +797,7 @@ public class ResetConsumerGroupOffsetTest { long expectedOffset) { return topics.stream() .collect(toMap(Function.identity(), - topic -> singletonMap(new TopicPartition(topic, 0), + topic -> Map.of(new TopicPartition(topic, 0), expectedOffset))); } @@ -822,7 +817,7 @@ public class ResetConsumerGroupOffsetTest { private static List generateIds(String name) { return IntStream.rangeClosed(1, 2) .mapToObj(id -> name + id) - .collect(Collectors.toList()); + .toList(); } private void produceConsumeAndShutdown(ClusterInstance cluster, @@ -877,7 +872,7 @@ public class ResetConsumerGroupOffsetTest { String topic, String group, long count) throws Exception { - try (Admin admin = Admin.create(singletonMap(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { + try (Admin admin = Admin.create(Map.of(BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()))) { Supplier offsets = () -> { try { return admin.listConsumerGroupOffsets(group) @@ -926,8 +921,8 @@ public class ResetConsumerGroupOffsetTest { } private String[] addTo(String[] args, String... extra) { - List res = new ArrayList<>(asList(args)); - res.addAll(asList(extra)); + List res = new ArrayList<>(List.of(args)); + res.addAll(List.of(extra)); return res.toArray(new String[0]); } } diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java index 1dc368c5070..3676cf4d482 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java @@ -39,9 +39,10 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.io.IOException; import java.time.Duration; -import java.util.Arrays; -import java.util.Collections; +import java.util.List; +import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -60,14 +61,14 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { public static final int NUM_PARTITIONS = 1; public static final int BROKER_COUNT = 1; public static final String KAFKA_CLIENT_SASL_MECHANISM = "SCRAM-SHA-256"; - private static final Seq KAFKA_SERVER_SASL_MECHANISMS = CollectionConverters.asScala(Collections.singletonList(KAFKA_CLIENT_SASL_MECHANISM)).toSeq(); + private static final Seq KAFKA_SERVER_SASL_MECHANISMS = CollectionConverters.asScala(List.of(KAFKA_CLIENT_SASL_MECHANISM)).toSeq(); private Consumer createConsumer() { return createConsumer( new ByteArrayDeserializer(), new ByteArrayDeserializer(), new Properties(), - CollectionConverters.asScala(Collections.emptySet()).toList() + CollectionConverters.asScala(Set.of()).toList() ); } @@ -99,7 +100,7 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { @Override public void addFormatterSettings(Formatter formatter) { formatter.setClusterId("XcZZOzUqS4yHOjhMQB6JLQ"); - formatter.setScramArguments(Arrays.asList("SCRAM-SHA-256=[name=" + JaasTestUtils.KAFKA_SCRAM_ADMIN + + formatter.setScramArguments(List.of("SCRAM-SHA-256=[name=" + JaasTestUtils.KAFKA_SCRAM_ADMIN + ",password=" + JaasTestUtils.KAFKA_SCRAM_ADMIN_PASSWORD + "]")); } @@ -119,7 +120,7 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { this.superuserClientConfig().put(SaslConfigs.SASL_JAAS_CONFIG, superuserLoginContext); super.setUp(testInfo); try (Admin admin = createPrivilegedAdminClient()) { - admin.createTopics(Collections.singletonList( + admin.createTopics(List.of( new NewTopic(TOPIC, NUM_PARTITIONS, (short) BROKER_COUNT))).all(). get(5, TimeUnit.MINUTES); } catch (ExecutionException | InterruptedException | TimeoutException e) { @@ -142,7 +143,7 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService(); Consumer consumer = createConsumer() ) { - consumer.subscribe(Collections.singletonList(TOPIC)); + consumer.subscribe(List.of(TOPIC)); verifyAuthenticationException(consumerGroupService::listGroups); } } @@ -156,7 +157,7 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService(); Consumer consumer = createConsumer() ) { - consumer.subscribe(Collections.singletonList(TOPIC)); + consumer.subscribe(List.of(TOPIC)); TestUtils.waitForCondition(() -> { try { @@ -180,7 +181,7 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { "--group", "test.group", "--command-config", propsFile.getAbsolutePath()}; ConsumerGroupCommandOptions opts = ConsumerGroupCommandOptions.fromArgs(cgcArgs); - return new ConsumerGroupCommand.ConsumerGroupService(opts, Collections.emptyMap()); + return new ConsumerGroupCommand.ConsumerGroupService(opts, Map.of()); } private void verifyAuthenticationException(Executable action) { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java index e80071907e7..b14d66c652a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ShareGroupCommandTest.java @@ -127,7 +127,7 @@ public class ShareGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--list"}; Admin adminClient = mock(KafkaAdminClient.class); ListGroupsResult result = mock(ListGroupsResult.class); - when(result.all()).thenReturn(KafkaFuture.completedFuture(Arrays.asList( + when(result.all()).thenReturn(KafkaFuture.completedFuture(List.of( new GroupListing(firstGroup, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.STABLE)), new GroupListing(secondGroup, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.EMPTY)) ))); @@ -153,7 +153,7 @@ public class ShareGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServer, "--list", "--state"}; Admin adminClient = mock(KafkaAdminClient.class); ListGroupsResult resultWithAllStates = mock(ListGroupsResult.class); - when(resultWithAllStates.all()).thenReturn(KafkaFuture.completedFuture(Arrays.asList( + when(resultWithAllStates.all()).thenReturn(KafkaFuture.completedFuture(List.of( new GroupListing(firstGroup, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.STABLE)), new GroupListing(secondGroup, Optional.of(GroupType.SHARE), "share", Optional.of(GroupState.EMPTY)) ))); @@ -571,7 +571,7 @@ public class ShareGroupCommandTest { for (List describeType : DESCRIBE_TYPES) { // note the group to be queried is a different (non-existing) group - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--describe", "--group", missingGroup)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", bootstrapServer, "--describe", "--group", missingGroup)); cgcArgs.addAll(describeType); Admin adminClient = mock(KafkaAdminClient.class); DescribeShareGroupsResult describeShareGroupsResult = mock(DescribeShareGroupsResult.class); @@ -677,7 +677,7 @@ public class ShareGroupCommandTest { String secondTopic = "t2"; String bootstrapServer = "localhost:9092"; - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); Admin adminClient = mock(KafkaAdminClient.class); DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); @@ -715,7 +715,7 @@ public class ShareGroupCommandTest { String secondTopic = "t2"; String bootstrapServer = "localhost:9092"; - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--group", secondGroup, "--topic", firstTopic, "--topic", secondTopic)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--group", secondGroup, "--topic", firstTopic, "--topic", secondTopic)); Admin adminClient = mock(KafkaAdminClient.class); try (ShareGroupService service = getShareGroupService(cgcArgs.toArray(new String[0]), adminClient)) { @@ -734,7 +734,7 @@ public class ShareGroupCommandTest { String secondTopic = "t2"; String bootstrapServer = "localhost:9092"; - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); Admin adminClient = mock(KafkaAdminClient.class); DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); @@ -787,7 +787,7 @@ public class ShareGroupCommandTest { String secondTopic = "t2"; String bootstrapServer = "localhost:9092"; - List cgcArgs = new ArrayList<>(Arrays.asList("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); + List cgcArgs = new ArrayList<>(List.of("--bootstrap-server", bootstrapServer, "--delete-offsets", "--group", firstGroup, "--topic", firstTopic, "--topic", secondTopic)); Admin adminClient = mock(KafkaAdminClient.class); DeleteShareGroupOffsetsResult result = mock(DeleteShareGroupOffsetsResult.class); diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/share/ShareGroupStateMessageFormatterTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/share/ShareGroupStateMessageFormatterTest.java index 0e452376786..31c6326ae4c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/share/ShareGroupStateMessageFormatterTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/share/ShareGroupStateMessageFormatterTest.java @@ -39,11 +39,11 @@ import org.junit.jupiter.params.provider.MethodSource; import java.io.ByteArrayOutputStream; import java.io.PrintStream; -import java.util.Arrays; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.stream.Stream; -import static java.util.Collections.emptyMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -55,7 +55,7 @@ public class ShareGroupStateMessageFormatterTest extends CoordinatorRecordMessag .setLeaderEpoch(20) .setStartOffset(50) .setStateBatches( - Arrays.asList( + List.of( new PersisterStateBatch( 100, 200, @@ -78,7 +78,7 @@ public class ShareGroupStateMessageFormatterTest extends CoordinatorRecordMessag .setLeaderEpoch(25) .setStartOffset(55) .setStateBatches( - Arrays.asList( + List.of( new PersisterStateBatch( 100, 150, @@ -229,7 +229,7 @@ public class ShareGroupStateMessageFormatterTest extends CoordinatorRecordMessag new RecordHeaders(), Optional.empty()); try (MessageFormatter formatter = new ShareGroupStateMessageFormatter()) { - formatter.configure(emptyMap()); + formatter.configure(Map.of()); ByteArrayOutputStream out = new ByteArrayOutputStream(); RuntimeException re = assertThrows(RuntimeException.class, () -> formatter.writeTo(record, new PrintStream(out))); assertEquals(expectedOutput.getMessage(), re.getMessage()); diff --git a/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java b/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java index bd708febb08..c2294be8ed5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java +++ b/tools/src/test/java/org/apache/kafka/tools/other/ReplicationQuotasTestRig.java @@ -59,11 +59,11 @@ import java.nio.file.Files; import java.text.DecimalFormat; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ExecutionException; import java.util.function.Function; @@ -106,7 +106,7 @@ public class ReplicationQuotasTestRig { boolean displayChartsOnScreen = args.length > 0 && Objects.equals(args[0], "show-gui"); Journal journal = new Journal(); - List experiments = Arrays.asList( + List experiments = List.of( //1GB total data written, will take 210s new ExperimentDef("Experiment1", 5, 20, 1 * K, 500, 100 * 1000), //5GB total data written, will take 110s @@ -209,11 +209,11 @@ public class ReplicationQuotasTestRig { Map> replicas = IntStream.rangeClosed(0, config.partitions - 1).boxed().collect(Collectors.toMap( Function.identity(), - partition -> Collections.singletonList(nextReplicaRoundRobin.getAsInt()) + partition -> List.of(nextReplicaRoundRobin.getAsInt()) )); startBrokers(config.brokers); - adminClient.createTopics(Collections.singleton(new NewTopic(TOPIC_NAME, replicas))).all().get(); + adminClient.createTopics(Set.of(new NewTopic(TOPIC_NAME, replicas))).all().get(); TestUtils.waitUntilTrue( () -> cluster.brokers().values().stream().allMatch(server -> { @@ -248,7 +248,7 @@ public class ReplicationQuotasTestRig { long start = System.currentTimeMillis(); ReassignPartitionsCommand.executeAssignment(adminClient, false, - ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Collections.emptyMap()), + ReassignPartitionsCommand.formatAsReassignmentJson(newAssignment, Map.of()), config.throttle, -1L, 10000L, Time.SYSTEM, false); //Await completion @@ -282,12 +282,12 @@ public class ReplicationQuotasTestRig { } void logOutput(ExperimentDef config, Map> replicas, Map> newAssignment) throws Exception { - List actual = adminClient.describeTopics(Collections.singleton(TOPIC_NAME)) + List actual = adminClient.describeTopics(Set.of(TOPIC_NAME)) .allTopicNames().get().get(TOPIC_NAME).partitions(); Map> curAssignment = actual.stream().collect(Collectors.toMap( TopicPartitionInfo::partition, - p -> p.replicas().stream().map(Node::id).collect(Collectors.toList()) + p -> p.replicas().stream().map(Node::id).toList() )); //Long stats diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index 381adc4699b..f1726721bfd 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -153,13 +153,35 @@ public class ReassignPartitionsCommandTest { produceMessages(foo0.topic(), foo0.partition(), 100); try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - String assignment = "{\"version\":1,\"partitions\":" + - "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[3,1,2],\"log_dirs\":[\"any\",\"any\",\"any\"]}" + - "]}"; - generateAssignment(admin, assignment, "1,2,3", false); + String topicsToMoveJson = """ + { + "topics": [ + { "topic": "foo" } + ], + "version": 1 + } + """; + var assignment = generateAssignment(admin, topicsToMoveJson, "1,2,3", false); + Map> proposedAssignments = assignment.getKey(); + String assignmentJson = String.format(""" + { + "version": 1, + "partitions": [ + { + "topic": "foo", + "partition": 0, + "replicas": %s, + "log_dirs": ["any", "any", "any"] + } + ] + } + """, proposedAssignments.get(foo0)); + + runExecuteAssignment(false, assignmentJson, -1L, -1L); + Map finalAssignment = Map.of(foo0, - new PartitionReassignmentState(List.of(0, 1, 2), List.of(3, 1, 2), true)); - waitForVerifyAssignment(admin, assignment, false, + new PartitionReassignmentState(proposedAssignments.get(foo0), proposedAssignments.get(foo0), true)); + waitForVerifyAssignment(admin, assignmentJson, false, new VerifyAssignmentResult(finalAssignment)); } } @@ -237,15 +259,15 @@ public class ReassignPartitionsCommandTest { // Check the reassignment status. VerifyAssignmentResult result = runVerifyAssignment(admin, assignment, true); - if (!result.partsOngoing) { + if (!result.partsOngoing()) { return true; } else { assertFalse( - result.partStates.values().stream().allMatch(state -> state.done), + result.partStates().values().stream().allMatch(PartitionReassignmentState::done), "Expected at least one partition reassignment to be ongoing when result = " + result ); - assertEquals(List.of(0, 3, 2), result.partStates.get(new TopicPartition("foo", 0)).targetReplicas); - assertEquals(List.of(3, 2, 1), result.partStates.get(new TopicPartition("baz", 2)).targetReplicas); + assertEquals(List.of(0, 3, 2), result.partStates().get(new TopicPartition("foo", 0)).targetReplicas()); + assertEquals(List.of(3, 2, 1), result.partStates().get(new TopicPartition("baz", 2)).targetReplicas()); waitForInterBrokerThrottle(admin, List.of(0, 1, 2, 3), interBrokerThrottle); return false; } @@ -427,7 +449,7 @@ public class ReassignPartitionsCommandTest { new CompletedMoveState(reassignment.targetDir) ), false)); - BrokerDirs info1 = new BrokerDirs(admin.describeLogDirs(IntStream.range(0, 4).boxed().collect(Collectors.toList())), 0); + BrokerDirs info1 = new BrokerDirs(admin.describeLogDirs(IntStream.range(0, 4).boxed().toList()), 0); assertEquals(reassignment.targetDir, info1.curLogDirs.getOrDefault(topicPartition, "")); } } @@ -540,7 +562,7 @@ public class ReassignPartitionsCommandTest { finalAssignment.put(bar0, new PartitionReassignmentState(List.of(3, 2, 0), List.of(3, 2, 0), true)); VerifyAssignmentResult verifyAssignmentResult = runVerifyAssignment(admin, assignment, false); - assertFalse(verifyAssignmentResult.movesOngoing); + assertFalse(verifyAssignmentResult.movesOngoing()); // Wait for the assignment to complete waitForVerifyAssignment(admin, assignment, false, @@ -640,8 +662,7 @@ public class ReassignPartitionsCommandTest { int brokerId, List replicas) throws ExecutionException, InterruptedException { try (Admin admin = Admin.create(Map.of(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, clusterInstance.bootstrapServers()))) { - DescribeLogDirsResult describeLogDirsResult = admin.describeLogDirs( - IntStream.range(0, 4).boxed().collect(Collectors.toList())); + DescribeLogDirsResult describeLogDirsResult = admin.describeLogDirs(IntStream.range(0, 4).boxed().toList()); BrokerDirs logDirInfo = new BrokerDirs(describeLogDirsResult, brokerId); assertTrue(logDirInfo.futureLogDirs.isEmpty()); @@ -654,7 +675,7 @@ public class ReassignPartitionsCommandTest { return "\"" + newDir + "\""; else return "\"any\""; - }).collect(Collectors.toList()); + }).toList(); String reassignmentJson = " { \"version\": 1," + @@ -787,7 +808,7 @@ public class ReassignPartitionsCommandTest { // This time, the broker throttles were removed. waitForBrokerLevelThrottles(admin, unthrottledBrokerConfigs); // Verify that there are no ongoing reassignments. - assertFalse(runVerifyAssignment(admin, assignment, false).partsOngoing); + assertFalse(runVerifyAssignment(admin, assignment, false).partsOngoing()); } // Verify that the partition is removed from cancelled replicas verifyReplicaDeleted(new TopicPartitionReplica(foo0.topic(), foo0.partition(), 3)); diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java index 77799ae0cf0..949b595a115 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsUnitTest.java @@ -40,6 +40,7 @@ import org.junit.jupiter.api.Timeout; import java.util.AbstractMap.SimpleImmutableEntry; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -51,7 +52,6 @@ import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; import java.util.stream.Stream; -import static java.util.Arrays.asList; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.alterPartitionReassignments; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.alterReplicaLogDirs; import static org.apache.kafka.tools.reassign.ReassignPartitionsCommand.calculateFollowerThrottles; @@ -120,11 +120,11 @@ public class ReassignPartitionsUnitTest { Map states = new HashMap<>(); states.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); + new PartitionReassignmentState(List.of(1, 2, 3), List.of(1, 2, 3), true)); states.put(new TopicPartition("foo", 1), - new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 4), false)); + new PartitionReassignmentState(List.of(1, 2, 3), List.of(1, 2, 4), false)); states.put(new TopicPartition("bar", 0), - new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 4), false)); + new PartitionReassignmentState(List.of(1, 2, 3), List.of(1, 2, 4), false)); assertEquals(String.join(System.lineSeparator(), "Status of partition reassignment:", @@ -136,19 +136,19 @@ public class ReassignPartitionsUnitTest { private void addTopics(MockAdminClient adminClient) { List b = adminClient.brokers(); - adminClient.addTopic(false, "foo", asList( + adminClient.addTopic(false, "foo", List.of( new TopicPartitionInfo(0, b.get(0), - asList(b.get(0), b.get(1), b.get(2)), - asList(b.get(0), b.get(1))), + List.of(b.get(0), b.get(1), b.get(2)), + List.of(b.get(0), b.get(1))), new TopicPartitionInfo(1, b.get(1), - asList(b.get(1), b.get(2), b.get(3)), - asList(b.get(1), b.get(2), b.get(3))) - ), Collections.emptyMap()); - adminClient.addTopic(false, "bar", asList( + List.of(b.get(1), b.get(2), b.get(3)), + List.of(b.get(1), b.get(2), b.get(3))) + ), Map.of()); + adminClient.addTopic(false, "bar", List.of( new TopicPartitionInfo(0, b.get(2), - asList(b.get(2), b.get(3), b.get(0)), - asList(b.get(2), b.get(3), b.get(0))) - ), Collections.emptyMap()); + List.of(b.get(2), b.get(3), b.get(0)), + List.of(b.get(2), b.get(3), b.get(0))) + ), Map.of()); } @Test @@ -158,8 +158,8 @@ public class ReassignPartitionsUnitTest { // Create a reassignment and test findPartitionReassignmentStates. Map> reassignments = new HashMap<>(); - reassignments.put(new TopicPartition("foo", 0), asList(0, 1, 3)); - reassignments.put(new TopicPartition("quux", 0), asList(1, 2, 3)); + reassignments.put(new TopicPartition("foo", 0), List.of(0, 1, 3)); + reassignments.put(new TopicPartition("quux", 0), List.of(1, 2, 3)); Map reassignmentResult = alterPartitionReassignments(adminClient, reassignments, false); @@ -169,14 +169,14 @@ public class ReassignPartitionsUnitTest { Map expStates = new HashMap<>(); expStates.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 3), false)); + new PartitionReassignmentState(List.of(0, 1, 2), List.of(0, 1, 3), false)); expStates.put(new TopicPartition("foo", 1), - new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); + new PartitionReassignmentState(List.of(1, 2, 3), List.of(1, 2, 3), true)); Entry, Boolean> actual = - findPartitionReassignmentStates(adminClient, asList( - new SimpleImmutableEntry<>(new TopicPartition("foo", 0), asList(0, 1, 3)), - new SimpleImmutableEntry<>(new TopicPartition("foo", 1), asList(1, 2, 3)) + findPartitionReassignmentStates(adminClient, List.of( + new SimpleImmutableEntry<>(new TopicPartition("foo", 0), List.of(0, 1, 3)), + new SimpleImmutableEntry<>(new TopicPartition("foo", 1), List.of(1, 2, 3)) )); assertEquals(expStates, actual.getKey()); @@ -192,13 +192,13 @@ public class ReassignPartitionsUnitTest { expStates.clear(); expStates.put(new TopicPartition("foo", 0), - new PartitionReassignmentState(asList(0, 1, 2), asList(0, 1, 3), true)); + new PartitionReassignmentState(List.of(0, 1, 2), List.of(0, 1, 3), true)); expStates.put(new TopicPartition("foo", 1), - new PartitionReassignmentState(asList(1, 2, 3), asList(1, 2, 3), true)); + new PartitionReassignmentState(List.of(1, 2, 3), List.of(1, 2, 3), true)); - actual = findPartitionReassignmentStates(adminClient, asList( - new SimpleImmutableEntry<>(new TopicPartition("foo", 0), asList(0, 1, 3)), - new SimpleImmutableEntry<>(new TopicPartition("foo", 1), asList(1, 2, 3)) + actual = findPartitionReassignmentStates(adminClient, List.of( + new SimpleImmutableEntry<>(new TopicPartition("foo", 0), List.of(0, 1, 3)), + new SimpleImmutableEntry<>(new TopicPartition("foo", 1), List.of(1, 2, 3)) )); assertEquals(expStates, actual.getKey()); @@ -210,20 +210,20 @@ public class ReassignPartitionsUnitTest { public void testFindLogDirMoveStates() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). numBrokers(4). - brokerLogDirs(asList( - asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"), - asList("/tmp/kafka-logs0", null))) + brokerLogDirs(List.of( + List.of("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + List.of("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + List.of("/tmp/kafka-logs0", "/tmp/kafka-logs1"), + Arrays.asList("/tmp/kafka-logs0", null))) .build()) { addTopics(adminClient); List b = adminClient.brokers(); - adminClient.addTopic(false, "quux", asList( + adminClient.addTopic(false, "quux", List.of( new TopicPartitionInfo(0, b.get(2), - asList(b.get(1), b.get(2), b.get(3)), - asList(b.get(1), b.get(2), b.get(3)))), - Collections.emptyMap()); + List.of(b.get(1), b.get(2), b.get(3)), + List.of(b.get(1), b.get(2), b.get(3)))), + Map.of()); Map replicaAssignment = new HashMap<>(); @@ -286,15 +286,15 @@ public class ReassignPartitionsUnitTest { Map> assignments = new HashMap<>(); - assignments.put(new TopicPartition("foo", 0), asList(0, 1, 2)); - assignments.put(new TopicPartition("foo", 1), asList(1, 2, 3)); + assignments.put(new TopicPartition("foo", 0), List.of(0, 1, 2)); + assignments.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); - assertEquals(assignments, getReplicaAssignmentForTopics(adminClient, asList("foo"))); + assertEquals(assignments, getReplicaAssignmentForTopics(adminClient, List.of("foo"))); assignments.clear(); - assignments.put(new TopicPartition("foo", 0), asList(0, 1, 2)); - assignments.put(new TopicPartition("bar", 0), asList(2, 3, 0)); + assignments.put(new TopicPartition("foo", 0), List.of(0, 1, 2)); + assignments.put(new TopicPartition("bar", 0), List.of(2, 3, 0)); assertEquals(assignments, getReplicaAssignmentForPartitions(adminClient, Set.of(new TopicPartition("foo", 0), new TopicPartition("bar", 0)))); @@ -311,26 +311,26 @@ public class ReassignPartitionsUnitTest { @Test public void testGetBrokerRackInformation() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). - brokers(asList(new Node(0, "localhost", 9092, "rack0"), + brokers(List.of(new Node(0, "localhost", 9092, "rack0"), new Node(1, "localhost", 9093, "rack1"), new Node(2, "localhost", 9094, null))). build()) { - assertEquals(asList( + assertEquals(List.of( new UsableBroker(0, Optional.of("rack0"), false), new UsableBroker(1, Optional.of("rack1"), false) - ), getBrokerMetadata(adminClient, asList(0, 1), true)); - assertEquals(asList( + ), getBrokerMetadata(adminClient, List.of(0, 1), true)); + assertEquals(List.of( new UsableBroker(0, Optional.empty(), false), new UsableBroker(1, Optional.empty(), false) - ), getBrokerMetadata(adminClient, asList(0, 1), false)); + ), getBrokerMetadata(adminClient, List.of(0, 1), false)); assertStartsWith("Not all brokers have rack information", assertThrows(AdminOperationException.class, - () -> getBrokerMetadata(adminClient, asList(1, 2), true)).getMessage()); - assertEquals(asList( + () -> getBrokerMetadata(adminClient, List.of(1, 2), true)).getMessage()); + assertEquals(List.of( new UsableBroker(1, Optional.empty(), false), new UsableBroker(2, Optional.empty(), false) - ), getBrokerMetadata(adminClient, asList(1, 2), false)); + ), getBrokerMetadata(adminClient, List.of(1, 2), false)); } } @@ -344,13 +344,13 @@ public class ReassignPartitionsUnitTest { assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4,5"), "Expected to detect duplicate broker list entries").getMessage()); - assertEquals(new SimpleImmutableEntry<>(asList(5, 2, 3, 4), asList("foo")), + assertEquals(new SimpleImmutableEntry<>(List.of(5, 2, 3, 4), List.of("foo")), parseGenerateAssignmentArgs("{\"topics\": [{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4")); assertStartsWith("List of topics to reassign contains duplicate entries", assertThrows(AdminCommandFailedException.class, () -> parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"foo\"}], \"version\":1}", "5,2,3,4"), "Expected to detect duplicate topic entries").getMessage()); - assertEquals(new SimpleImmutableEntry<>(asList(5, 3, 4), asList("foo", "bar")), + assertEquals(new SimpleImmutableEntry<>(List.of(5, 3, 4), List.of("foo", "bar")), parseGenerateAssignmentArgs( "{\"topics\": [{\"topic\": \"foo\"},{\"topic\": \"bar\"}], \"version\":1}", "5,3,4")); } @@ -380,7 +380,7 @@ public class ReassignPartitionsUnitTest { @Test public void testGenerateAssignmentWithInconsistentRacks() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder(). - brokers(asList( + brokers(List.of( new Node(0, "localhost", 9092, "rack0"), new Node(1, "localhost", 9093, "rack0"), new Node(2, "localhost", 9094, null), @@ -400,8 +400,8 @@ public class ReassignPartitionsUnitTest { Map> expCurrent = new HashMap<>(); - expCurrent.put(new TopicPartition("foo", 0), asList(0, 1, 2)); - expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3)); + expCurrent.put(new TopicPartition("foo", 0), List.of(0, 1, 2)); + expCurrent.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); assertEquals(expCurrent, proposedCurrent.getValue()); } @@ -411,7 +411,7 @@ public class ReassignPartitionsUnitTest { public void testGenerateAssignmentWithFewerBrokers() throws Exception { try (MockAdminClient adminClient = new MockAdminClient.Builder().numBrokers(4).build()) { addTopics(adminClient); - List goalBrokers = asList(0, 1, 3); + List goalBrokers = List.of(0, 1, 3); Entry>, Map>> proposedCurrent = generateAssignment(adminClient, @@ -420,9 +420,9 @@ public class ReassignPartitionsUnitTest { Map> expCurrent = new HashMap<>(); - expCurrent.put(new TopicPartition("foo", 0), asList(0, 1, 2)); - expCurrent.put(new TopicPartition("foo", 1), asList(1, 2, 3)); - expCurrent.put(new TopicPartition("bar", 0), asList(2, 3, 0)); + expCurrent.put(new TopicPartition("foo", 0), List.of(0, 1, 2)); + expCurrent.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); + expCurrent.put(new TopicPartition("bar", 0), List.of(2, 3, 0)); assertEquals(expCurrent, proposedCurrent.getValue()); @@ -438,15 +438,15 @@ public class ReassignPartitionsUnitTest { public void testCurrentPartitionReplicaAssignmentToString() throws Exception { Map> proposedParts = new HashMap<>(); - proposedParts.put(new TopicPartition("foo", 1), asList(1, 2, 3)); - proposedParts.put(new TopicPartition("bar", 0), asList(7, 8, 9)); + proposedParts.put(new TopicPartition("foo", 1), List.of(1, 2, 3)); + proposedParts.put(new TopicPartition("bar", 0), List.of(7, 8, 9)); Map> currentParts = new HashMap<>(); - currentParts.put(new TopicPartition("foo", 0), asList(1, 2, 3)); - currentParts.put(new TopicPartition("foo", 1), asList(4, 5, 6)); - currentParts.put(new TopicPartition("bar", 0), asList(7, 8)); - currentParts.put(new TopicPartition("baz", 0), asList(10, 11, 12)); + currentParts.put(new TopicPartition("foo", 0), List.of(1, 2, 3)); + currentParts.put(new TopicPartition("foo", 1), List.of(4, 5, 6)); + currentParts.put(new TopicPartition("bar", 0), List.of(7, 8)); + currentParts.put(new TopicPartition("baz", 0), List.of(10, 11, 12)); assertEquals(String.join(System.lineSeparator(), "Current partition replica assignment", @@ -473,37 +473,37 @@ public class ReassignPartitionsUnitTest { Map currentReassignments = new HashMap<>(); currentReassignments.put(new TopicPartition("foo", 0), new PartitionReassignment( - asList(1, 2, 3, 4), asList(4), asList(3))); + List.of(1, 2, 3, 4), List.of(4), List.of(3))); currentReassignments.put(new TopicPartition("foo", 1), new PartitionReassignment( - asList(4, 5, 6, 7, 8), asList(7, 8), asList(4, 5))); + List.of(4, 5, 6, 7, 8), List.of(7, 8), List.of(4, 5))); currentReassignments.put(new TopicPartition("foo", 2), new PartitionReassignment( - asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); + List.of(1, 2, 3, 4), List.of(3, 4), List.of(1, 2))); currentReassignments.put(new TopicPartition("foo", 3), new PartitionReassignment( - asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); + List.of(1, 2, 3, 4), List.of(3, 4), List.of(1, 2))); currentReassignments.put(new TopicPartition("foo", 4), new PartitionReassignment( - asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); + List.of(1, 2, 3, 4), List.of(3, 4), List.of(1, 2))); currentReassignments.put(new TopicPartition("foo", 5), new PartitionReassignment( - asList(1, 2, 3, 4), asList(3, 4), asList(1, 2))); + List.of(1, 2, 3, 4), List.of(3, 4), List.of(1, 2))); Map> proposedParts = new HashMap<>(); - proposedParts.put(new TopicPartition("foo", 0), asList(1, 2, 5)); - proposedParts.put(new TopicPartition("foo", 2), asList(3, 4)); - proposedParts.put(new TopicPartition("foo", 3), asList(5, 6)); - proposedParts.put(new TopicPartition("foo", 4), asList(3)); - proposedParts.put(new TopicPartition("foo", 5), asList(3, 4, 5, 6)); - proposedParts.put(new TopicPartition("bar", 0), asList(1, 2, 3)); + proposedParts.put(new TopicPartition("foo", 0), List.of(1, 2, 5)); + proposedParts.put(new TopicPartition("foo", 2), List.of(3, 4)); + proposedParts.put(new TopicPartition("foo", 3), List.of(5, 6)); + proposedParts.put(new TopicPartition("foo", 4), List.of(3)); + proposedParts.put(new TopicPartition("foo", 5), List.of(3, 4, 5, 6)); + proposedParts.put(new TopicPartition("bar", 0), List.of(1, 2, 3)); Map> currentParts = new HashMap<>(); - currentParts.put(new TopicPartition("foo", 0), asList(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 1), asList(4, 5, 6, 7, 8)); - currentParts.put(new TopicPartition("foo", 2), asList(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 3), asList(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 4), asList(1, 2, 3, 4)); - currentParts.put(new TopicPartition("foo", 5), asList(1, 2, 3, 4)); - currentParts.put(new TopicPartition("bar", 0), asList(2, 3, 4)); - currentParts.put(new TopicPartition("baz", 0), asList(1, 2, 3)); + currentParts.put(new TopicPartition("foo", 0), List.of(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 1), List.of(4, 5, 6, 7, 8)); + currentParts.put(new TopicPartition("foo", 2), List.of(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 3), List.of(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 4), List.of(1, 2, 3, 4)); + currentParts.put(new TopicPartition("foo", 5), List.of(1, 2, 3, 4)); + currentParts.put(new TopicPartition("bar", 0), List.of(2, 3, 4)); + currentParts.put(new TopicPartition("baz", 0), List.of(1, 2, 3)); Map> moveMap = calculateProposedMoveMap(currentReassignments, proposedParts, currentParts); @@ -570,8 +570,8 @@ public class ReassignPartitionsUnitTest { Map> partitionsToBeReassigned = new HashMap<>(); - partitionsToBeReassigned.put(new TopicPartition("foo", 0), asList(1, 2, 3)); - partitionsToBeReassigned.put(new TopicPartition("foo", 1), asList(3, 4, 5)); + partitionsToBeReassigned.put(new TopicPartition("foo", 0), List.of(1, 2, 3)); + partitionsToBeReassigned.put(new TopicPartition("foo", 1), List.of(3, 4, 5)); Entry>, Map> actual = parseExecuteAssignmentArgs( "{\"version\":1,\"partitions\":" + @@ -593,7 +593,7 @@ public class ReassignPartitionsUnitTest { "[{\"topic\":\"foo\",\"partition\":0,\"replicas\":[1,2,3],\"log_dirs\":[\"/tmp/a\",\"/tmp/b\",\"/tmp/c\"]}" + "]}"); - assertEquals(Collections.singletonMap(new TopicPartition("foo", 0), asList(1, 2, 3)), actual.getKey()); + assertEquals(Map.of(new TopicPartition("foo", 0), List.of(1, 2, 3)), actual.getKey()); assertEquals(replicaAssignment, actual.getValue()); } @@ -666,9 +666,9 @@ public class ReassignPartitionsUnitTest { Map> reassignments = new HashMap<>(); - reassignments.put(new TopicPartition("foo", 1), asList(4, 5, 3)); - reassignments.put(new TopicPartition("foo", 0), asList(0, 1, 4, 2)); - reassignments.put(new TopicPartition("bar", 0), asList(2, 3)); + reassignments.put(new TopicPartition("foo", 1), List.of(4, 5, 3)); + reassignments.put(new TopicPartition("foo", 0), List.of(0, 1, 4, 2)); + reassignments.put(new TopicPartition("bar", 0), List.of(2, 3)); Map reassignmentResult = alterPartitionReassignments(adminClient, reassignments, false); @@ -711,9 +711,9 @@ public class ReassignPartitionsUnitTest { modifyTopicThrottles(adminClient, leaderThrottles, - Collections.singletonMap("bar", "followerBar")); + Map.of("bar", "followerBar")); List topics = Stream.of("bar", "foo").map( - id -> new ConfigResource(ConfigResource.Type.TOPIC, id)).collect(Collectors.toList()); + id -> new ConfigResource(ConfigResource.Type.TOPIC, id)).toList(); Map results = adminClient.describeConfigs(topics).all().get(); verifyTopicThrottleResults(results.get(topics.get(0)), "leaderBar", "followerBar"); verifyTopicThrottleResults(results.get(topics.get(1)), "leaderFoo", ""); @@ -736,7 +736,7 @@ public class ReassignPartitionsUnitTest { try (MockAdminClient adminClient = new MockAdminClient.Builder(). numBrokers(4). brokerLogDirs(Collections.nCopies(4, - asList("/tmp/kafka-logs0", "/tmp/kafka-logs1"))). + List.of("/tmp/kafka-logs0", "/tmp/kafka-logs1"))). build()) { addTopics(adminClient); diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupOffsetTest.java index 16e2770c87f..94e9b30acfe 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupOffsetTest.java @@ -50,7 +50,6 @@ import org.junit.jupiter.api.Timeout; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -409,7 +408,7 @@ public class DeleteStreamsGroupOffsetTest { private static StreamsBuilder builder(String inputTopic, String outputTopic) { final StreamsBuilder builder = new StreamsBuilder(); builder.stream(inputTopic, Consumed.with(Serdes.String(), Serdes.String())) - .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .flatMapValues(value -> List.of(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy((key, value) -> value) .count() .toStream().to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupTest.java index aa89d8796de..25994e60a57 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/DeleteStreamsGroupTest.java @@ -55,7 +55,6 @@ import org.junit.jupiter.api.Timeout; import java.time.Duration; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -560,7 +559,7 @@ public class DeleteStreamsGroupTest { private static StreamsBuilder builder(String inputTopic, String outputTopic) { final StreamsBuilder builder = new StreamsBuilder(); builder.stream(inputTopic, Consumed.with(Serdes.String(), Serdes.String())) - .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .flatMapValues(value -> List.of(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy((key, value) -> value) .count() .toStream().to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java index 92c2f036f77..9c4d4016748 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/DescribeStreamsGroupTest.java @@ -129,10 +129,10 @@ public class DescribeStreamsGroupTest { List.of(APP_ID, "streams-group-command-test-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition", "1", "0")); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--group", APP_ID), expectedHeader, expectedRows, List.of()); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--group", APP_ID), expectedHeader, expectedRows, List.of()); // --describe --offsets has the same output as --describe validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--offsets", "--group", APP_ID), expectedHeader, expectedRows, List.of()); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--offsets", "--group", APP_ID), expectedHeader, expectedRows, List.of()); } @Test @@ -145,35 +145,35 @@ public class DescribeStreamsGroupTest { List.of(APP_ID, "streams-group-command-test-KSTREAM-AGGREGATE-STATE-STORE-0000000003-repartition", "1", "-", "-", "0", "0")); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, List.of()); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, List.of()); // --describe --offsets has the same output as --describe validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--offsets", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, List.of()); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--offsets", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, List.of()); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--offsets", "--group", APP_ID), expectedHeader, expectedRows, List.of()); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--offsets", "--group", APP_ID), expectedHeader, expectedRows, List.of()); } @Test public void testDescribeStreamsGroupWithStateOption() throws Exception { - final List expectedHeader = Arrays.asList("GROUP", "COORDINATOR", "(ID)", "STATE", "#MEMBERS"); - final Set> expectedRows = Set.of(Arrays.asList(APP_ID, "", "", "Stable", "2")); + final List expectedHeader = List.of("GROUP", "COORDINATOR", "(ID)", "STATE", "#MEMBERS"); + final Set> expectedRows = Set.of(List.of(APP_ID, "", "", "Stable", "2")); // The coordinator is not deterministic, so we don't care about it. final List dontCares = List.of(1, 2); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--state", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--state", "--group", APP_ID), expectedHeader, expectedRows, dontCares); } @Test public void testDescribeStreamsGroupWithStateAndVerboseOptions() throws Exception { - final List expectedHeader = Arrays.asList("GROUP", "COORDINATOR", "(ID)", "STATE", "GROUP-EPOCH", "TARGET-ASSIGNMENT-EPOCH", "#MEMBERS"); - final Set> expectedRows = Set.of(Arrays.asList(APP_ID, "", "", "Stable", "3", "3", "2")); + final List expectedHeader = List.of("GROUP", "COORDINATOR", "(ID)", "STATE", "GROUP-EPOCH", "TARGET-ASSIGNMENT-EPOCH", "#MEMBERS"); + final Set> expectedRows = Set.of(List.of(APP_ID, "", "", "Stable", "3", "3", "2")); // The coordinator is not deterministic, so we don't care about it. final List dontCares = List.of(1, 2); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--state", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--state", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, dontCares); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--state", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--state", "--group", APP_ID), expectedHeader, expectedRows, dontCares); } @Test @@ -186,7 +186,7 @@ public class DescribeStreamsGroupTest { final List dontCares = List.of(1, 2, 3); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--members", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--members", "--group", APP_ID), expectedHeader, expectedRows, dontCares); } @Test @@ -199,9 +199,9 @@ public class DescribeStreamsGroupTest { final List dontCares = List.of(3, 6, 7); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID), expectedHeader, expectedRows, dontCares); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", APP_ID), expectedHeader, expectedRows, dontCares); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", APP_ID), expectedHeader, expectedRows, dontCares); } @Test @@ -225,13 +225,13 @@ public class DescribeStreamsGroupTest { final List dontCares = List.of(3, 6, 7); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID, "--group", APP_ID_2), + List.of("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID, "--group", APP_ID_2), expectedHeader, expectedRowsMap, dontCares); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", APP_ID, "--group", APP_ID_2), + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", APP_ID, "--group", APP_ID_2), expectedHeader, expectedRowsMap, dontCares); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--all-groups"), + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--all-groups"), expectedHeader, expectedRowsMap, dontCares); streams2.close(); @@ -246,14 +246,14 @@ public class DescribeStreamsGroupTest { nonExistingGroup); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", nonExistingGroup), errorMessage); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", nonExistingGroup), errorMessage); validateDescribeOutput( - Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", nonExistingGroup), errorMessage); + List.of("--bootstrap-server", bootstrapServers, "--describe", "--verbose", "--members", "--group", nonExistingGroup), errorMessage); } @Test public void testDescribeStreamsGroupWithShortTimeout() { - List args = Arrays.asList("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID, "--timeout", "1"); + List args = List.of("--bootstrap-server", bootstrapServers, "--describe", "--members", "--verbose", "--group", APP_ID, "--timeout", "1"); Throwable e = assertThrows(ExecutionException.class, () -> getStreamsGroupService(args.toArray(new String[0])).describeGroups()); assertEquals(TimeoutException.class, e.getCause().getClass()); } @@ -261,7 +261,7 @@ public class DescribeStreamsGroupTest { private static Topology topology(String inputTopic, String outputTopic) { final StreamsBuilder builder = new StreamsBuilder(); builder.stream(inputTopic, Consumed.with(Serdes.String(), Serdes.String())) - .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .flatMapValues(value -> List.of(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy((key, value) -> value) .count() .toStream().to(outputTopic, Produced.with(Serdes.String(), Serdes.Long())); @@ -309,11 +309,11 @@ public class DescribeStreamsGroupTest { if (lines.length == 1 && lines[0].isEmpty()) lines = new String[]{}; if (lines.length == 0) return false; - List header = Arrays.asList(lines[0].split("\\s+")); + List header = List.of(lines[0].split("\\s+")); if (!expectedHeader.equals(header)) return false; Set> groupDesc = Arrays.stream(Arrays.copyOfRange(lines, 1, lines.length)) - .map(line -> Arrays.asList(line.split("\\s+"))) + .map(line -> List.of(line.split("\\s+"))) .collect(Collectors.toSet()); if (groupDesc.size() != expectedRows.size()) return false; // clear the dontCare fields and then compare two sets @@ -344,7 +344,7 @@ public class DescribeStreamsGroupTest { if (lines.length == 1 && lines[0].isEmpty()) lines = new String[]{}; if (lines.length == 0) return false; - List header = Arrays.asList(lines[0].split("\\s+")); + List header = List.of(lines[0].split("\\s+")); if (!expectedHeader.equals(header)) return false; Map>> groupdescMap = splitOutputByGroup(lines); @@ -382,7 +382,7 @@ public class DescribeStreamsGroupTest { if (lines[i].replaceAll(" ", "").equals(headerLine) || i == lines.length - 1) { if (i == lines.length - 1) i++; Set> groupDesc = Arrays.stream(Arrays.copyOfRange(lines, j, i)) - .map(line -> Arrays.asList(line.split("\\s+"))) + .map(line -> List.of(line.split("\\s+"))) .collect(Collectors.toSet()); groupdescMap.put(groupName, groupDesc); if (i + 1 < lines.length) { diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/ListStreamsGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/ListStreamsGroupTest.java index dee821f71c1..fcb971d35c7 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/ListStreamsGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/ListStreamsGroupTest.java @@ -42,10 +42,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Objects; import java.util.Optional; import java.util.Properties; @@ -133,7 +133,7 @@ public class ListStreamsGroupTest { final AtomicReference> foundListing = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Collections.emptySet()))); + foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Set.of()))); return Objects.equals(expectedListing, foundListing.get()); }, "Expected --list to show streams groups " + expectedListing + ", but found " + foundListing.get() + "."); } @@ -153,18 +153,18 @@ public class ListStreamsGroupTest { final AtomicReference> foundListing = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Collections.emptySet()))); + foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Set.of()))); return Objects.equals(expectedListing, foundListing.get()); }, "Expected --list to show streams groups " + expectedListing + ", but found " + foundListing.get() + "."); } try (StreamsGroupCommand.StreamsGroupService service = getStreamsGroupService(new String[]{"--bootstrap-server", cluster.bootstrapServers(), "--list", "--state", "PreparingRebalance"})) { - Set expectedListing = Collections.emptySet(); + Set expectedListing = Set.of(); final AtomicReference> foundListing = new AtomicReference<>(); TestUtils.waitForCondition(() -> { - foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Collections.singleton(GroupState.PREPARING_REBALANCE)))); + foundListing.set(new HashSet<>(service.listStreamsGroupsInStates(Set.of(GroupState.PREPARING_REBALANCE)))); return Objects.equals(expectedListing, foundListing.get()); }, "Expected --list to show streams groups " + expectedListing + ", but found " + foundListing.get() + "."); } @@ -173,35 +173,35 @@ public class ListStreamsGroupTest { @Test public void testListStreamsGroupOutput() throws Exception { validateListOutput( - Arrays.asList("--bootstrap-server", cluster.bootstrapServers(), "--list"), - Collections.emptyList(), - Set.of(Collections.singletonList(APP_ID)) + List.of("--bootstrap-server", cluster.bootstrapServers(), "--list"), + List.of(), + Set.of(List.of(APP_ID)) ); validateListOutput( - Arrays.asList("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state"), - Arrays.asList("GROUP", "STATE"), - Set.of(Arrays.asList(APP_ID, "Stable")) + List.of("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state"), + List.of("GROUP", "STATE"), + Set.of(List.of(APP_ID, "Stable")) ); validateListOutput( - Arrays.asList("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state", "Stable"), - Arrays.asList("GROUP", "STATE"), - Set.of(Arrays.asList(APP_ID, "Stable")) + List.of("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state", "Stable"), + List.of("GROUP", "STATE"), + Set.of(List.of(APP_ID, "Stable")) ); // Check case-insensitivity in state filter. validateListOutput( - Arrays.asList("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state", "stable"), - Arrays.asList("GROUP", "STATE"), - Set.of(Arrays.asList(APP_ID, "Stable")) + List.of("--bootstrap-server", cluster.bootstrapServers(), "--list", "--state", "stable"), + List.of("GROUP", "STATE"), + Set.of(List.of(APP_ID, "Stable")) ); } private static Topology topology() { final StreamsBuilder builder = new StreamsBuilder(); builder.stream(INPUT_TOPIC, Consumed.with(Serdes.String(), Serdes.String())) - .flatMapValues(value -> Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+"))) + .flatMapValues(value -> List.of(value.toLowerCase(Locale.getDefault()).split("\\W+"))) .groupBy((key, value) -> value) .count() .toStream().to(OUTPUT_TOPIC, Produced.with(Serdes.String(), Serdes.Long())); @@ -212,7 +212,7 @@ public class ListStreamsGroupTest { StreamsGroupCommandOptions opts = StreamsGroupCommandOptions.fromArgs(args); return new StreamsGroupCommand.StreamsGroupService( opts, - Collections.singletonMap(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) + Map.of(AdminClientConfig.RETRIES_CONFIG, Integer.toString(Integer.MAX_VALUE)) ); } @@ -230,12 +230,12 @@ public class ListStreamsGroupTest { if (lines.length == 1 && lines[0].isEmpty()) lines = new String[]{}; if (!expectedHeader.isEmpty() && lines.length > 0) { - List header = Arrays.asList(lines[0].split("\\s+")); + List header = List.of(lines[0].split("\\s+")); if (!expectedHeader.equals(header)) return false; } Set> groups = Arrays.stream(lines, expectedHeader.isEmpty() ? 0 : 1, lines.length) - .map(line -> Arrays.asList(line.split("\\s+"))) + .map(line -> List.of(line.split("\\s+"))) .collect(Collectors.toSet()); return expectedRows.equals(groups); }, () -> String.format("Expected header=%s and groups=%s, but found:%n%s", expectedHeader, expectedRows, out.get())); diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/ResetStreamsGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/ResetStreamsGroupOffsetTest.java index 8a843538632..2838e496b59 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/ResetStreamsGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/ResetStreamsGroupOffsetTest.java @@ -70,7 +70,6 @@ import java.util.stream.Collectors; import joptsimple.OptionException; import static java.time.LocalDateTime.now; -import static java.util.Arrays.asList; import static java.util.stream.Collectors.toMap; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -443,13 +442,13 @@ public class ResetStreamsGroupOffsetTest { } // assert that the reset offsets are as expected assertEquals(expectedResetResults, resetOffsetsResultByGroup); - assertEquals(expectedResetResults.values().size(), resetOffsetsResultByGroup.values().size()); + assertEquals(expectedResetResults.size(), resetOffsetsResultByGroup.size()); // assert that the committed offsets are as expected AssertCommittedOffsets(appId, topic, expectedCommittedOffset, partitions); } private void resetOffsetsAndAssertInternalTopicDeletion(String[] args, String appId, String... specifiedInternalTopics) throws InterruptedException { - List specifiedInternalTopicsList = asList(specifiedInternalTopics); + List specifiedInternalTopicsList = List.of(specifiedInternalTopics); Set allInternalTopics = getInternalTopics(appId); specifiedInternalTopicsList.forEach(allInternalTopics::remove); @@ -465,10 +464,10 @@ public class ResetStreamsGroupOffsetTest { ); // verify that the specified internal topics were deleted Set internalTopicsAfterReset = getInternalTopics(appId); - specifiedInternalTopicsList.forEach(topic -> { + specifiedInternalTopicsList.forEach(topic -> assertFalse(internalTopicsAfterReset.contains(topic), - "Internal topic '" + topic + "' was not deleted as expected after reset"); - }); + "Internal topic '" + topic + "' was not deleted as expected after reset") + ); } else { TestUtils.waitForCondition(() -> { @@ -530,7 +529,7 @@ public class ResetStreamsGroupOffsetTest { } // assert that the reset offsets are as expected assertEquals(expectedResetResults, resetOffsetsResultByGroup); - assertEquals(expectedResetResults.values().size(), resetOffsetsResultByGroup.values().size()); + assertEquals(expectedResetResults.size(), resetOffsetsResultByGroup.size()); // assert that the committed offsets are as expected AssertCommittedOffsets(appId, topic1, topic2, expectedCommittedOffset); } @@ -565,7 +564,7 @@ public class ResetStreamsGroupOffsetTest { } // assert that the reset offsets are as expected assertEquals(expectedOffsets, resetOffsetsResult); - assertEquals(expectedOffsets.values().size(), resetOffsetsResult.values().size()); + assertEquals(expectedOffsets.size(), resetOffsetsResult.size()); // assert that the committed offsets are as expected assertEquals(expectedCommittedOffsets, committedOffsets(topics, appId)); } @@ -631,8 +630,8 @@ public class ResetStreamsGroupOffsetTest { } private String[] addTo(String[] args, String... extra) { - List res = new ArrayList<>(asList(args)); - res.addAll(asList(extra)); + List res = new ArrayList<>(List.of(args)); + res.addAll(List.of(extra)); return res.toArray(new String[0]); } diff --git a/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java index bf1296dd235..6f38c47f15a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/streams/StreamsGroupCommandTest.java @@ -52,7 +52,6 @@ import org.mockito.ArgumentMatchers; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -91,13 +90,13 @@ public class StreamsGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", BOOTSTRAP_SERVERS, "--list"}; ListGroupsResult result = mock(ListGroupsResult.class); - when(result.all()).thenReturn(KafkaFuture.completedFuture(Arrays.asList( + when(result.all()).thenReturn(KafkaFuture.completedFuture(List.of( new GroupListing(firstGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)), new GroupListing(secondGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.EMPTY)) ))); when(ADMIN_CLIENT.listGroups(any(ListGroupsOptions.class))).thenReturn(result); StreamsGroupCommand.StreamsGroupService service = getStreamsGroupService(cgcArgs); - Set expectedGroups = new HashSet<>(Arrays.asList(firstGroup, secondGroup)); + Set expectedGroups = Set.of(firstGroup, secondGroup); final Set[] foundGroups = new Set[]{Set.of()}; TestUtils.waitForCondition(() -> { @@ -111,9 +110,7 @@ public class StreamsGroupCommandTest { @Test public void testListWithUnrecognizedOption() { String[] cgcArgs = new String[]{"--frivolous-nonsense", "--bootstrap-server", BOOTSTRAP_SERVERS, "--list"}; - final Exception exception = assertThrows(OptionException.class, () -> { - getStreamsGroupService(cgcArgs); - }); + final Exception exception = assertThrows(OptionException.class, () -> getStreamsGroupService(cgcArgs)); assertEquals("frivolous-nonsense is not a recognized option", exception.getMessage()); } @@ -124,19 +121,19 @@ public class StreamsGroupCommandTest { String[] cgcArgs = new String[]{"--bootstrap-server", BOOTSTRAP_SERVERS, "--list", "--state"}; ListGroupsResult resultWithAllStates = mock(ListGroupsResult.class); - when(resultWithAllStates.all()).thenReturn(KafkaFuture.completedFuture(Arrays.asList( + when(resultWithAllStates.all()).thenReturn(KafkaFuture.completedFuture(List.of( new GroupListing(firstGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)), new GroupListing(secondGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.EMPTY)) ))); when(ADMIN_CLIENT.listGroups(any(ListGroupsOptions.class))).thenReturn(resultWithAllStates); StreamsGroupCommand.StreamsGroupService service = getStreamsGroupService(cgcArgs); - Set expectedListing = new HashSet<>(Arrays.asList( + Set expectedListing = Set.of( new GroupListing(firstGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.STABLE)), - new GroupListing(secondGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.EMPTY)))); + new GroupListing(secondGroup, Optional.of(GroupType.STREAMS), "streams", Optional.of(GroupState.EMPTY))); final Set[] foundListing = new Set[]{Set.of()}; TestUtils.waitForCondition(() -> { - foundListing[0] = new HashSet<>(service.listStreamsGroupsInStates(new HashSet<>(Arrays.asList(GroupState.values())))); + foundListing[0] = new HashSet<>(service.listStreamsGroupsInStates(Set.of(GroupState.values()))); return Objects.equals(expectedListing, foundListing[0]); }, "Expected to show groups " + expectedListing + ", but found " + foundListing[0]); @@ -257,34 +254,34 @@ public class StreamsGroupCommandTest { @Test public void testGroupStatesFromString() { Set result = StreamsGroupCommand.groupStatesFromString("empty"); - assertEquals(new HashSet<>(List.of(GroupState.EMPTY)), result); + assertEquals(Set.of(GroupState.EMPTY), result); result = StreamsGroupCommand.groupStatesFromString("EMPTY"); - assertEquals(new HashSet<>(List.of(GroupState.EMPTY)), result); + assertEquals(Set.of(GroupState.EMPTY), result); result = StreamsGroupCommand.groupStatesFromString("notready"); - assertEquals(new HashSet<>(List.of(GroupState.NOT_READY)), result); + assertEquals(Set.of(GroupState.NOT_READY), result); result = StreamsGroupCommand.groupStatesFromString("notReady"); - assertEquals(new HashSet<>(List.of(GroupState.NOT_READY)), result); + assertEquals(Set.of(GroupState.NOT_READY), result); result = StreamsGroupCommand.groupStatesFromString("assigning"); - assertEquals(new HashSet<>(List.of(GroupState.ASSIGNING)), result); + assertEquals(Set.of(GroupState.ASSIGNING), result); result = StreamsGroupCommand.groupStatesFromString("ASSIGNING"); - assertEquals(new HashSet<>(List.of(GroupState.ASSIGNING)), result); + assertEquals(Set.of(GroupState.ASSIGNING), result); result = StreamsGroupCommand.groupStatesFromString("RECONCILING"); - assertEquals(new HashSet<>(List.of(GroupState.RECONCILING)), result); + assertEquals(Set.of(GroupState.RECONCILING), result); result = StreamsGroupCommand.groupStatesFromString("reconCILING"); - assertEquals(new HashSet<>(List.of(GroupState.RECONCILING)), result); + assertEquals(Set.of(GroupState.RECONCILING), result); result = StreamsGroupCommand.groupStatesFromString("STABLE"); - assertEquals(new HashSet<>(List.of(GroupState.STABLE)), result); + assertEquals(Set.of(GroupState.STABLE), result); result = StreamsGroupCommand.groupStatesFromString("stable"); - assertEquals(new HashSet<>(List.of(GroupState.STABLE)), result); + assertEquals(Set.of(GroupState.STABLE), result); result = StreamsGroupCommand.groupStatesFromString("DEAD"); - assertEquals(new HashSet<>(List.of(GroupState.DEAD)), result); + assertEquals(Set.of(GroupState.DEAD), result); result = StreamsGroupCommand.groupStatesFromString("dead"); - assertEquals(new HashSet<>(List.of(GroupState.DEAD)), result); + assertEquals(Set.of(GroupState.DEAD), result); assertThrow("preparingRebalance"); assertThrow("completingRebalance"); @@ -297,7 +294,7 @@ public class StreamsGroupCommandTest { public void testAdminRequestsForResetOffsets() { Admin adminClient = mock(KafkaAdminClient.class); String groupId = "foo-group"; - List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", groupId, "--reset-offsets", "--input-topic", "topic1", "--to-latest")); + List args = List.of("--bootstrap-server", "localhost:9092", "--group", groupId, "--reset-offsets", "--input-topic", "topic1", "--to-latest"); List topics = List.of("topic1"); when(adminClient.describeStreamsGroups(List.of(groupId))) @@ -317,8 +314,8 @@ public class StreamsGroupCommandTest { StreamsGroupCommand.StreamsGroupService service = getStreamsGroupService(args.toArray(new String[0]), adminClient); Map> resetResult = service.resetOffsets(); - assertEquals(Collections.singleton(groupId), resetResult.keySet()); - assertEquals(new HashSet<>(List.of(new TopicPartition(topics.get(0), 0))), + assertEquals(Set.of(groupId), resetResult.keySet()); + assertEquals(Set.of(new TopicPartition(topics.get(0), 0)), resetResult.get(groupId).keySet()); verify(adminClient, times(1)).describeStreamsGroups(List.of(groupId)); @@ -332,7 +329,7 @@ public class StreamsGroupCommandTest { @Test public void testRetrieveInternalTopics() { String groupId = "foo-group"; - List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete")); + List args = List.of("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete"); List sourceTopics = List.of("source-topic1", "source-topic2"); List repartitionSinkTopics = List.of("rep-sink-topic1", "rep-sink-topic2"); Map stateChangelogTopics = Map.of( @@ -365,7 +362,7 @@ public class StreamsGroupCommandTest { assertNotNull(internalTopics.get(groupId)); assertEquals(4, internalTopics.get(groupId).size()); - assertEquals(new HashSet<>(List.of(groupId + "-1-changelog", groupId + "-2-changelog", groupId + "-1-repartition", groupId + "-2-repartition")), + assertEquals(Set.of(groupId + "-1-changelog", groupId + "-2-changelog", groupId + "-1-repartition", groupId + "-2-repartition"), new HashSet<>(internalTopics.get(groupId))); assertFalse(internalTopics.get(groupId).stream().anyMatch(List.of("some-pre-fix-changelog", groupId + "-some-thing")::contains)); assertFalse(internalTopics.get(groupId).stream().anyMatch(sourceTopics::contains)); @@ -378,7 +375,7 @@ public class StreamsGroupCommandTest { public void testDeleteStreamsGroup() { Admin adminClient = mock(KafkaAdminClient.class); String groupId = "foo-group"; - List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete", "--delete-all-internal-topics")); + List args = List.of("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete", "--delete-all-internal-topics"); DeleteStreamsGroupsResult deleteStreamsGroupsResult = mock(DeleteStreamsGroupsResult.class); when(adminClient.deleteStreamsGroups(eq(List.of(groupId)), any(DeleteStreamsGroupsOptions.class))).thenReturn(deleteStreamsGroupsResult); @@ -409,7 +406,7 @@ public class StreamsGroupCommandTest { public void testDeleteNonStreamsGroup() { Admin adminClient = mock(KafkaAdminClient.class); String groupId = "foo-group"; - List args = new ArrayList<>(Arrays.asList("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete")); + List args = List.of("--bootstrap-server", "localhost:9092", "--group", groupId, "--delete"); ListGroupsResult listGroupsResult = mock(ListGroupsResult.class); when(adminClient.listGroups(any())).thenReturn(listGroupsResult); @@ -419,8 +416,8 @@ public class StreamsGroupCommandTest { Map result = service.deleteGroups(); assertNotNull(result.get(groupId)); - assertEquals(result.get(groupId).getMessage(), - "Group '" + groupId + "' does not exist or is not a streams group."); + assertEquals("Group '" + groupId + "' does not exist or is not a streams group.", + result.get(groupId).getMessage()); assertInstanceOf(IllegalArgumentException.class, result.get(groupId)); verify(adminClient, times(1)).listGroups(any(ListGroupsOptions.class)); // we do not expect any further API to be called @@ -450,7 +447,7 @@ public class StreamsGroupCommandTest { } private static void assertThrow(final String wrongState) { - final Set validStates = new HashSet<>(Arrays.asList("Assigning", "Dead", "Empty", "Reconciling", "Stable", "NotReady")); + final Set validStates = Set.of("Assigning", "Dead", "Empty", "Reconciling", "Stable", "NotReady"); final Exception exception = assertThrows(IllegalArgumentException.class, () -> StreamsGroupCommand.groupStatesFromString(wrongState)); @@ -473,25 +470,21 @@ public class StreamsGroupCommandTest { 0, 0, 0, - Collections.singletonList(new StreamsGroupSubtopologyDescription("subtopologyId", Collections.emptyList(), Collections.emptyList(), Map.of(), Map.of())), + List.of(new StreamsGroupSubtopologyDescription("subtopologyId", List.of(), List.of(), Map.of(), Map.of())), List.of(memberDescription), groupState, new Node(1, "localhost", 9092), Set.of()); KafkaFutureImpl future = new KafkaFutureImpl<>(); future.complete(description); - return new DescribeStreamsGroupsResult(Collections.singletonMap(groupId, future)); + return new DescribeStreamsGroupsResult(Map.of(groupId, future)); } private DescribeTopicsResult describeTopicsResult(Collection topics, int numOfPartitions) { - Map topicDescriptions = new HashMap<>(); - - topics.forEach(topic -> { - List partitions = IntStream.range(0, numOfPartitions) - .mapToObj(i -> new TopicPartitionInfo(i, null, Collections.emptyList(), Collections.emptyList())) - .collect(Collectors.toList()); - topicDescriptions.put(topic, new TopicDescription(topic, false, partitions)); - }); + var topicDescriptions = topics.stream().collect(Collectors.toMap(Function.identity(), + topic -> new TopicDescription(topic, false, IntStream.range(0, numOfPartitions) + .mapToObj(i -> new TopicPartitionInfo(i, null, List.of(), List.of())) + .toList()))); return AdminClientTestUtils.describeTopicsResult(topicDescriptions); } diff --git a/tools/tools-api/src/test/java/org/apache/kafka/tools/api/RecordReaderTest.java b/tools/tools-api/src/test/java/org/apache/kafka/tools/api/RecordReaderTest.java index 4c2145d7ffe..716c638986e 100644 --- a/tools/tools-api/src/test/java/org/apache/kafka/tools/api/RecordReaderTest.java +++ b/tools/tools-api/src/test/java/org/apache/kafka/tools/api/RecordReaderTest.java @@ -19,7 +19,7 @@ package org.apache.kafka.tools.api; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import java.util.Collections; +import java.util.Map; public class RecordReaderTest { @@ -27,7 +27,7 @@ public class RecordReaderTest { void testDefaultCloseAndConfigure() { RecordReader reader = inputStream -> null; // `configure` and `close` should have default empty body - Assertions.assertDoesNotThrow(() -> reader.configure(Collections.emptyMap())); + Assertions.assertDoesNotThrow(() -> reader.configure(Map.of())); Assertions.assertDoesNotThrow(reader::close); } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java index fb3a7d8162f..e0f729a7931 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/agent/AgentClient.java @@ -45,7 +45,6 @@ import org.slf4j.LoggerFactory; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; @@ -275,9 +274,7 @@ public class AgentClient { System.out.printf("\tStart time: %s%n", dateString(status.serverStartMs(), localOffset)); List> lines = new ArrayList<>(); - List header = new ArrayList<>( - Arrays.asList("WORKER_ID", "TASK_ID", "STATE", "TASK_TYPE")); - lines.add(header); + lines.add(List.of("WORKER_ID", "TASK_ID", "STATE", "TASK_TYPE")); for (Map.Entry entry : status.workers().entrySet()) { List cols = new ArrayList<>(); cols.add(Long.toString(entry.getKey())); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java index 729031f0b84..3fb0379b8fb 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/basic/BasicNode.java @@ -21,7 +21,6 @@ import org.apache.kafka.trogdor.common.Node; import com.fasterxml.jackson.databind.JsonNode; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -46,7 +45,7 @@ public class BasicNode implements Node { public BasicNode(String name, JsonNode root) { this.name = name; String hostname = "localhost"; - Set tags = Collections.emptySet(); + Set tags = new HashSet<>(); Map config = new HashMap<>(); for (Map.Entry entry : root.properties()) { String key = entry.getKey(); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java b/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java index 35ca5955052..072fcf8fa63 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/common/StringFormatter.java @@ -102,9 +102,7 @@ public class StringFormatter { String val = cols.get(x); int minWidth = widths.get(x); bld.append(val); - for (int i = 0; i < minWidth - val.length(); i++) { - bld.append(" "); - } + bld.append(" ".repeat(Math.max(0, minWidth - val.length()))); } bld.append(String.format("%n")); } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java index d5e14e959d3..b1e67bda5b4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/CoordinatorClient.java @@ -55,7 +55,6 @@ import org.slf4j.LoggerFactory; import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Optional; @@ -471,9 +470,7 @@ public class CoordinatorClient { return "No matching tasks found."; } List> lines = new ArrayList<>(); - List header = new ArrayList<>( - Arrays.asList("ID", "TYPE", "STATE", "INFO")); - lines.add(header); + lines.add(List.of("ID", "TYPE", "STATE", "INFO")); for (Map.Entry entry : response.tasks().entrySet()) { String taskId = entry.getKey(); TaskState taskState = entry.getValue(); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java index 03e78796bc8..6b10de71d48 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/coordinator/TaskManager.java @@ -256,17 +256,12 @@ public final class TaskManager { } TaskState taskState() { - switch (state) { - case PENDING: - return new TaskPending(spec); - case RUNNING: - return new TaskRunning(spec, startedMs, getCombinedStatus()); - case STOPPING: - return new TaskStopping(spec, startedMs, getCombinedStatus()); - case DONE: - return new TaskDone(spec, startedMs, doneMs, error, cancelled, getCombinedStatus()); - } - throw new RuntimeException("unreachable"); + return switch (state) { + case PENDING -> new TaskPending(spec); + case RUNNING -> new TaskRunning(spec, startedMs, getCombinedStatus()); + case STOPPING -> new TaskStopping(spec, startedMs, getCombinedStatus()); + case DONE -> new TaskDone(spec, startedMs, doneMs, error, cancelled, getCombinedStatus()); + }; } private JsonNode getCombinedStatus() { diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java index 59a2eb00eb0..d3153b3c7f2 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/DegradedNetworkFaultSpec.java @@ -24,7 +24,6 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; public class DegradedNetworkFaultSpec extends TaskSpec { @@ -75,7 +74,7 @@ public class DegradedNetworkFaultSpec extends TaskSpec { @JsonProperty("durationMs") long durationMs, @JsonProperty("nodeSpecs") Map nodeSpecs) { super(startMs, durationMs); - this.nodeSpecs = nodeSpecs == null ? Collections.emptyMap() : Collections.unmodifiableMap(nodeSpecs); + this.nodeSpecs = nodeSpecs == null ? Map.of() : Map.copyOf(nodeSpecs); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java index eb3de147228..862cb0f5b41 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/Kibosh.java @@ -29,7 +29,6 @@ import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Objects; import java.util.TreeMap; @@ -123,7 +122,7 @@ public final class Kibosh { private final List faults; public static final KiboshControlFile EMPTY = - new KiboshControlFile(Collections.emptyList()); + new KiboshControlFile(List.of()); public static KiboshControlFile read(Path controlPath) throws IOException { byte[] controlFileBytes = Files.readAllBytes(controlPath); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java index 9cb8b5724a2..331abd0038d 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/fault/NetworkPartitionFaultWorker.java @@ -72,9 +72,7 @@ public class NetworkPartitionFaultWorker implements TaskWorker { TreeSet toBlock = new TreeSet<>(); for (Set partitionSet : partitionSets) { if (!partitionSet.contains(curNode.name())) { - for (String nodeName : partitionSet) { - toBlock.add(nodeName); - } + toBlock.addAll(partitionSet); } } for (String nodeName : toBlock) { diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java index fa8df147b84..1936ea84787 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/ErrorResponse.java @@ -22,15 +22,10 @@ import org.apache.kafka.trogdor.common.JsonUtil; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Objects; - /** * An error response. */ -public class ErrorResponse { - private final int code; - private final String message; - +public record ErrorResponse(int code, String message) { @JsonCreator public ErrorResponse(@JsonProperty("code") int code, @JsonProperty("message") String message) { @@ -38,30 +33,18 @@ public class ErrorResponse { this.message = message; } + @Override @JsonProperty public int code() { return code; } + @Override @JsonProperty public String message() { return message; } - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - ErrorResponse that = (ErrorResponse) o; - return Objects.equals(code, that.code) && - Objects.equals(message, that.message); - } - - @Override - public int hashCode() { - return Objects.hash(code, message); - } - @Override public String toString() { return JsonUtil.toJsonString(this); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java index e42738f5587..1ad383d9324 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TaskRequest.java @@ -23,16 +23,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** * The request to /coordinator/tasks/{taskId} */ -public class TaskRequest { - private final String taskId; - +public record TaskRequest(String taskId) { @JsonCreator public TaskRequest(@JsonProperty("taskId") String taskId) { this.taskId = taskId == null ? "" : taskId; } - - @JsonProperty - public String taskId() { - return taskId; - } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java index 7ed3d7c8d65..0e33d8c3848 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksRequest.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; import java.util.Optional; import java.util.Set; @@ -69,8 +67,7 @@ public class TasksRequest extends Message { @JsonProperty("firstEndMs") long firstEndMs, @JsonProperty("lastEndMs") long lastEndMs, @JsonProperty("state") Optional state) { - this.taskIds = Collections.unmodifiableSet((taskIds == null) ? - new HashSet<>() : new HashSet<>(taskIds)); + this.taskIds = taskIds == null ? Set.of() : Set.copyOf(taskIds); this.firstStartMs = Math.max(0, firstStartMs); this.lastStartMs = Math.max(0, lastStartMs); this.firstEndMs = Math.max(0, firstEndMs); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java index 5f8655fd563..31e207bc7fa 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/rest/TasksResponse.java @@ -20,7 +20,6 @@ package org.apache.kafka.trogdor.rest; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; import java.util.TreeMap; @@ -32,7 +31,7 @@ public class TasksResponse extends Message { @JsonCreator public TasksResponse(@JsonProperty("tasks") TreeMap tasks) { - this.tasks = Collections.unmodifiableMap((tasks == null) ? new TreeMap<>() : tasks); + this.tasks = tasks == null ? Map.of() : Map.copyOf(tasks); } @JsonProperty diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java index f7a538195b8..a9585b95f73 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/task/TaskSpec.java @@ -22,7 +22,6 @@ import org.apache.kafka.trogdor.common.JsonUtil; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeInfo; -import java.util.Collections; import java.util.Map; import java.util.Objects; @@ -112,6 +111,6 @@ public abstract class TaskSpec { } protected static Map configOrEmptyMap(Map config) { - return (config == null) ? Collections.emptyMap() : config; + return config == null ? Map.of() : config; } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java index 8f6556842f5..aa4340ef2d3 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConfigurableProducerSpec.java @@ -24,9 +24,9 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; import java.util.Optional; +import java.util.Set; /** * This is the spec to pass in to be able to run the `ConfigurableProducerWorker` workload. This allows for customized @@ -206,7 +206,7 @@ public final class ConfigurableProducerSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(producerNode); + return topology -> Set.of(producerNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java index dc0b09da96a..20651f1ed0a 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressSpec.java @@ -24,7 +24,6 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeSet; @@ -56,8 +55,7 @@ public final class ConnectionStressSpec extends TaskSpec { @JsonProperty("numThreads") int numThreads, @JsonProperty("action") ConnectionStressAction action) { super(startMs, durationMs); - this.clientNodes = (clientNodes == null) ? Collections.emptyList() : - List.copyOf(clientNodes); + this.clientNodes = clientNodes == null ? List.of() : List.copyOf(clientNodes); this.bootstrapServers = (bootstrapServers == null) ? "" : bootstrapServers; this.commonClientConf = configOrEmptyMap(commonClientConf); this.targetConnectionsPerSec = targetConnectionsPerSec; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java index af861d38195..9452c03ec99 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConnectionStressWorker.java @@ -129,13 +129,10 @@ public class ConnectionStressWorker implements TaskWorker { interface Stressor extends AutoCloseable { static Stressor fromSpec(ConnectionStressSpec spec) { - switch (spec.action()) { - case CONNECT: - return new ConnectStressor(spec); - case FETCH_METADATA: - return new FetchMetadataStressor(spec); - } - throw new RuntimeException("invalid spec.action " + spec.action()); + return switch (spec.action()) { + case CONNECT -> new ConnectStressor(spec); + case FETCH_METADATA -> new FetchMetadataStressor(spec); + }; } boolean tryConnect(); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java index 24869e3e966..12dd1bb80a0 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpec.java @@ -29,7 +29,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -189,7 +188,7 @@ public final class ConsumeBenchSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(consumerNode); + return topology -> Set.of(consumerNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java index 9c5e053293f..0a51db48404 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ConsumeBenchWorker.java @@ -58,7 +58,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantLock; -import java.util.stream.Collectors; public class ConsumeBenchWorker implements TaskWorker { @@ -132,7 +131,7 @@ public class ConsumeBenchWorker implements TaskWorker { } } else { List partitions = populatePartitionsByTopic(consumer.consumer(), partitionsByTopic) - .values().stream().flatMap(List::stream).collect(Collectors.toList()); + .values().stream().flatMap(List::stream).toList(); tasks.add(new ConsumeMessages(consumer, spec.recordProcessor(), partitions)); for (int i = 0; i < consumerCount - 1; i++) { @@ -182,7 +181,7 @@ public class ConsumeBenchWorker implements TaskWorker { if (partitions.isEmpty()) { List fetchedPartitions = consumer.partitionsFor(topicName).stream() .map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition())) - .collect(Collectors.toList()); + .toList(); partitions.addAll(fetchedPartitions); } @@ -550,7 +549,7 @@ public class ConsumeBenchWorker implements TaskWorker { this.consumerLock.lock(); try { return consumer.assignment().stream() - .map(TopicPartition::toString).collect(Collectors.toList()); + .map(TopicPartition::toString).toList(); } finally { this.consumerLock.unlock(); } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java index 838beb62cff..dae242c22d0 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ExternalCommandSpec.java @@ -26,10 +26,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.NullNode; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Optional; +import java.util.Set; /** * ExternalCommandSpec describes a task that executes Trogdor tasks with the command. @@ -78,7 +77,7 @@ public class ExternalCommandSpec extends TaskSpec { @JsonProperty("shutdownGracePeriodMs") Optional shutdownGracePeriodMs) { super(startMs, durationMs); this.commandNode = (commandNode == null) ? "" : commandNode; - this.command = (command == null) ? Collections.unmodifiableList(new ArrayList<>()) : command; + this.command = (command == null) ? List.of() : command; this.workload = (workload == null) ? NullNode.instance : workload; this.shutdownGracePeriodMs = shutdownGracePeriodMs; } @@ -105,7 +104,7 @@ public class ExternalCommandSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(commandNode); + return topology -> Set.of(commandNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java index a8c24a446c1..33615eb5f15 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/Histogram.java @@ -18,7 +18,6 @@ package org.apache.kafka.trogdor.workload; import java.util.ArrayList; -import java.util.Collections; import java.util.List; /** @@ -175,7 +174,7 @@ public class Histogram { private List summarizePercentiles(int[] countsCopy, float[] percentiles, long numSamples) { if (percentiles.length == 0) { - return Collections.emptyList(); + return List.of(); } List summaries = new ArrayList<>(percentiles.length); int i = 0, j = 0; diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java index 9cb00efcbc6..2a2f27b470f 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/PartitionsSpec.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,12 +58,12 @@ public class PartitionsSpec extends Message { assignments.add(brokerId == null ? Integer.valueOf(0) : brokerId); } } - partMap.put(partition, Collections.unmodifiableList(assignments)); + partMap.put(partition, List.copyOf(assignments)); } } - this.partitionAssignments = Collections.unmodifiableMap(partMap); + this.partitionAssignments = Map.copyOf(partMap); if (configs == null) { - this.configs = Collections.emptyMap(); + this.configs = Map.of(); } else { this.configs = Map.copyOf(configs); } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java index 621333d74f9..f33855c94e4 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ProduceBenchSpec.java @@ -24,9 +24,9 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; import java.util.Optional; +import java.util.Set; /** * The specification for a benchmark that produces messages to a set of topics. @@ -185,7 +185,7 @@ public final class ProduceBenchSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(producerNode); + return topology -> Set.of(producerNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java index 5f5da5735de..69be574a264 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RandomComponent.java @@ -22,27 +22,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; /** * Contains a percent value represented as an integer between 1 and 100 and a PayloadGenerator to specify - * how often that PayloadGenerator should be used. + * how often that PayloadGenerator should be used. */ -public class RandomComponent { - private final int percent; - private final PayloadGenerator component; - - +public record RandomComponent(int percent, PayloadGenerator component) { @JsonCreator public RandomComponent(@JsonProperty("percent") int percent, @JsonProperty("component") PayloadGenerator component) { this.percent = percent; this.component = component; } - - @JsonProperty - public int percent() { - return percent; - } - - @JsonProperty - public PayloadGenerator component() { - return component; - } } diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java index 67fec60990b..fd8f5228995 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/RoundTripWorkloadSpec.java @@ -24,8 +24,8 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; +import java.util.Set; /** * The specification for a workload that sends messages to a broker and then @@ -123,7 +123,7 @@ public class RoundTripWorkloadSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(clientNode); + return topology -> Set.of(clientNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpec.java index 503d2b9b827..559121358d7 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpec.java @@ -27,7 +27,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -176,7 +175,7 @@ public final class ShareConsumeBenchSpec extends TaskSpec { @Override public TaskController newController(String id) { - return topology -> Collections.singleton(consumerNode); + return topology -> Set.of(consumerNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java index ccd94784f35..144ce3a2d45 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionSpec.java @@ -24,8 +24,8 @@ import org.apache.kafka.trogdor.task.TaskWorker; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; import java.util.Map; +import java.util.Set; /** * The specification for a benchmark that creates sustained connections. @@ -187,7 +187,7 @@ public final class SustainedConnectionSpec extends TaskSpec { } public TaskController newController(String id) { - return topology -> Collections.singleton(clientNode); + return topology -> Set.of(clientNode); } @Override diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java index 559a1885850..3e8c3c7fd14 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/SustainedConnectionWorker.java @@ -343,15 +343,15 @@ public class SustainedConnectionWorker implements TaskWorker { this.consumer = new KafkaConsumer<>(this.props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); List partitions = this.consumer.partitionsFor(this.topicName).stream() .map(partitionInfo -> new TopicPartition(partitionInfo.topic(), partitionInfo.partition())) - .collect(Collectors.toList()); + .toList(); // Select a random partition and assign it. this.activePartition = partitions.get(this.rand.nextInt(partitions.size())); - this.consumer.assign(Collections.singletonList(this.activePartition)); + this.consumer.assign(List.of(this.activePartition)); } // The behavior when passing in an empty list is to seek to the end of all subscribed partitions. - this.consumer.seekToEnd(Collections.emptyList()); + this.consumer.seekToEnd(List.of()); // Poll to keep the connection alive, ignoring any records returned. this.consumer.poll(Duration.ofMillis(50)); diff --git a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java index 600fd46f21f..693b80c143d 100644 --- a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java +++ b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TopicsSpec.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonAnyGetter; import com.fasterxml.jackson.annotation.JsonAnySetter; import com.fasterxml.jackson.annotation.JsonCreator; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -68,9 +67,7 @@ public class TopicsSpec extends Message { } public TopicsSpec immutableCopy() { - HashMap mapCopy = new HashMap<>(); - mapCopy.putAll(map); - return new TopicsSpec(Collections.unmodifiableMap(mapCopy)); + return new TopicsSpec(Map.copyOf(map)); } /** diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java index 070de473056..e21059823f3 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/agent/AgentTest.java @@ -61,11 +61,12 @@ import java.io.PrintStream; import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.nio.file.Paths; -import java.util.Collections; import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.TreeMap; -import static java.util.Arrays.asList; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -80,7 +81,7 @@ public class AgentTest { HashMap config = new HashMap<>(); config.put(Platform.Config.TROGDOR_AGENT_PORT, Integer.toString(Agent.DEFAULT_PORT)); nodes.put("node01", new BasicNode("node01", "localhost", - config, Collections.emptySet())); + config, Set.of())); BasicTopology topology = new BasicTopology(nodes); return new BasicPlatform("node01", topology, scheduler, new BasicPlatform.ShellCommandRunner()); @@ -153,7 +154,7 @@ public class AgentTest { maxTries(10).target("localhost", agent.port()).build(); AgentStatusResponse status = client.status(); - assertEquals(Collections.emptyMap(), status.workers()); + assertEquals(Map.of(), status.workers()); new ExpectedTasks().waitFor(client); final NoOpTaskSpec fooSpec = new NoOpTaskSpec(10, 10); @@ -191,7 +192,7 @@ public class AgentTest { AgentClient client = new AgentClient.Builder(). maxTries(10).target("localhost", agent.port()).build(); AgentStatusResponse status = client.status(); - assertEquals(Collections.emptyMap(), status.workers()); + assertEquals(Map.of(), status.workers()); new ExpectedTasks().waitFor(client); final NoOpTaskSpec fooSpec = new NoOpTaskSpec(1000, 600000); @@ -304,7 +305,7 @@ public class AgentTest { new ExpectedTasks().waitFor(client); SampleTaskSpec fooSpec = new SampleTaskSpec(0, 900000, - Collections.singletonMap("node01", 1L), ""); + Map.of("node01", 1L), ""); client.createWorker(new CreateWorkerRequest(0, "foo", fooSpec)); new ExpectedTasks(). addTask(new ExpectedTaskBuilder("foo"). @@ -313,7 +314,7 @@ public class AgentTest { waitFor(client); SampleTaskSpec barSpec = new SampleTaskSpec(0, 900000, - Collections.singletonMap("node01", 2L), "baz"); + Map.of("node01", 2L), "baz"); client.createWorker(new CreateWorkerRequest(1, "bar", barSpec)); time.sleep(1); @@ -373,17 +374,17 @@ public class AgentTest { try (MockKibosh mockKibosh = new MockKibosh()) { assertEquals(KiboshControlFile.EMPTY, mockKibosh.read()); FilesUnreadableFaultSpec fooSpec = new FilesUnreadableFaultSpec(0, 900000, - Collections.singleton("myAgent"), mockKibosh.tempDir.getPath(), "/foo", 123); + Set.of("myAgent"), mockKibosh.tempDir.getPath(), "/foo", 123); client.createWorker(new CreateWorkerRequest(0, "foo", fooSpec)); new ExpectedTasks(). addTask(new ExpectedTaskBuilder("foo"). workerState(new WorkerRunning("foo", fooSpec, 0, new TextNode("Added fault foo"))). build()). waitFor(client); - assertEquals(new KiboshControlFile(Collections.singletonList( + assertEquals(new KiboshControlFile(List.of( new KiboshFilesUnreadableFaultSpec("/foo", 123))), mockKibosh.read()); FilesUnreadableFaultSpec barSpec = new FilesUnreadableFaultSpec(0, 900000, - Collections.singleton("myAgent"), mockKibosh.tempDir.getPath(), "/bar", 456); + Set.of("myAgent"), mockKibosh.tempDir.getPath(), "/bar", 456); client.createWorker(new CreateWorkerRequest(1, "bar", barSpec)); new ExpectedTasks(). addTask(new ExpectedTaskBuilder("foo"). @@ -391,7 +392,7 @@ public class AgentTest { addTask(new ExpectedTaskBuilder("bar"). workerState(new WorkerRunning("bar", barSpec, 0, new TextNode("Added fault bar"))).build()). waitFor(client); - assertEquals(new KiboshControlFile(asList( + assertEquals(new KiboshControlFile(List.of( new KiboshFilesUnreadableFaultSpec("/foo", 123), new KiboshFilesUnreadableFaultSpec("/bar", 456)) ), mockKibosh.read()); @@ -403,7 +404,7 @@ public class AgentTest { addTask(new ExpectedTaskBuilder("bar"). workerState(new WorkerRunning("bar", barSpec, 0, new TextNode("Added fault bar"))).build()). waitFor(client); - assertEquals(new KiboshControlFile(Collections.singletonList( + assertEquals(new KiboshControlFile(List.of( new KiboshFilesUnreadableFaultSpec("/bar", 456))), mockKibosh.read()); } } @@ -476,7 +477,7 @@ public class AgentTest { public void testAgentExecWithNormalExit() throws Exception { Agent agent = createAgent(Scheduler.SYSTEM); SampleTaskSpec spec = new SampleTaskSpec(0, 120000, - Collections.singletonMap("node01", 1L), ""); + Map.of("node01", 1L), ""); TaskSpec rebasedSpec = agent.rebaseTaskSpecTime(spec); testExec(agent, String.format("Waiting for completion of task:%s%n", diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java index 93d5d9f3984..4eefe2e0263 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonSerializationTest.java @@ -35,7 +35,6 @@ import org.junit.jupiter.api.Test; import java.lang.reflect.Field; import java.lang.reflect.Modifier; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -63,8 +62,8 @@ public class JsonSerializationTest { verify(new TopicsSpec()); verify(new PartitionsSpec(0, (short) 0, null, null)); Map> partitionAssignments = new HashMap<>(); - partitionAssignments.put(0, Arrays.asList(1, 2, 3)); - partitionAssignments.put(1, Arrays.asList(1, 2, 3)); + partitionAssignments.put(0, List.of(1, 2, 3)); + partitionAssignments.put(1, List.of(1, 2, 3)); verify(new PartitionsSpec(0, (short) 0, partitionAssignments, null)); verify(new PartitionsSpec(0, (short) 0, null, null)); } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java index 6b7b67a894b..8f42d043311 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/JsonUtilTest.java @@ -45,10 +45,7 @@ public class JsonUtilTest { assertFalse(JsonUtil.openBraceComesFirst(" blah{}")); } - static final class Foo { - @JsonProperty - final int bar; - + record Foo(@JsonProperty int bar) { @JsonCreator Foo(@JsonProperty("bar") int bar) { this.bar = bar; diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java index 64e8c5def1c..e3cc7884d68 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/MiniTrogdorCluster.java @@ -33,9 +33,9 @@ import org.apache.kafka.trogdor.rest.JsonRestServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; import java.util.concurrent.Callable; @@ -160,8 +160,7 @@ public class MiniTrogdorCluster implements AutoCloseable { config.put(Platform.Config.TROGDOR_COORDINATOR_PORT, Integer.toString(node.coordinatorPort)); } - node.node = new BasicNode(entry.getKey(), node.hostname, config, - Collections.emptySet()); + node.node = new BasicNode(entry.getKey(), node.hostname, config, Set.of()); } TreeMap topologyNodes = new TreeMap<>(); for (Map.Entry entry : nodes.entrySet()) { diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java index 4ece85846ad..4c52a670960 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringExpanderTest.java @@ -20,7 +20,6 @@ package org.apache.kafka.trogdor.common; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Collections; import java.util.Set; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -30,9 +29,9 @@ public class StringExpanderTest { @Test public void testNoExpansionNeeded() { - assertEquals(Collections.singleton("foo"), StringExpander.expand("foo")); - assertEquals(Collections.singleton("bar"), StringExpander.expand("bar")); - assertEquals(Collections.singleton(""), StringExpander.expand("")); + assertEquals(Set.of("foo"), StringExpander.expand("foo")); + assertEquals(Set.of("bar"), StringExpander.expand("bar")); + assertEquals(Set.of(""), StringExpander.expand("")); } @Test diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java index 2c78f61aac0..b2a800d032f 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/StringFormatterTest.java @@ -21,7 +21,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; import java.time.ZoneOffset; -import java.util.Arrays; +import java.util.List; import static org.apache.kafka.trogdor.common.StringFormatter.dateString; import static org.apache.kafka.trogdor.common.StringFormatter.durationString; @@ -53,8 +53,8 @@ public class StringFormatterTest { "lion 1 12345 %n" + "manatee 50 1 %n"), StringFormatter.prettyPrintGrid( - Arrays.asList(Arrays.asList("ANIMAL", "NUMBER", "INDEX"), - Arrays.asList("lion", "1", "12345"), - Arrays.asList("manatee", "50", "1")))); + List.of(List.of("ANIMAL", "NUMBER", "INDEX"), + List.of("lion", "1", "12345"), + List.of("manatee", "50", "1")))); } } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java index 3ada5d75470..58affc48ca5 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/common/WorkerUtilsTest.java @@ -33,9 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -53,8 +51,8 @@ public class WorkerUtilsTest { private final Node broker1 = new Node(0, "testHost-1", 1234); private final Node broker2 = new Node(1, "testHost-2", 1234); private final Node broker3 = new Node(1, "testHost-3", 1234); - private final List cluster = Arrays.asList(broker1, broker2, broker3); - private final List singleReplica = Collections.singletonList(broker1); + private final List cluster = List.of(broker1, broker2, broker3); + private final List singleReplica = List.of(broker1); private static final String TEST_TOPIC = "test-topic-1"; private static final short TEST_REPLICATION_FACTOR = 1; @@ -72,17 +70,17 @@ public class WorkerUtilsTest { @Test public void testCreateOneTopic() throws Throwable { - Map newTopics = Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC); + Map newTopics = Map.of(TEST_TOPIC, NEW_TEST_TOPIC); WorkerUtils.createTopics(log, adminClient, newTopics, true); - assertEquals(Collections.singleton(TEST_TOPIC), adminClient.listTopics().names().get()); + assertEquals(Set.of(TEST_TOPIC), adminClient.listTopics().names().get()); assertEquals( new TopicDescription( TEST_TOPIC, false, - Collections.singletonList( - new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()))), + List.of( + new TopicPartitionInfo(0, broker1, singleReplica, List.of(), List.of(), List.of()))), adminClient.describeTopics( - Collections.singleton(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() + Set.of(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() ); } @@ -91,21 +89,21 @@ public class WorkerUtilsTest { adminClient.timeoutNextRequest(1); WorkerUtils.createTopics( - log, adminClient, Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC), true); + log, adminClient, Map.of(TEST_TOPIC, NEW_TEST_TOPIC), true); assertEquals( new TopicDescription( TEST_TOPIC, false, - Collections.singletonList( - new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()))), + List.of( + new TopicPartitionInfo(0, broker1, singleReplica, List.of(), List.of(), List.of()))), adminClient.describeTopics( - Collections.singleton(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() + Set.of(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() ); } @Test public void testCreateZeroTopicsDoesNothing() throws Throwable { - WorkerUtils.createTopics(log, adminClient, Collections.emptyMap(), true); + WorkerUtils.createTopics(log, adminClient, Map.of(), true); assertEquals(0, adminClient.listTopics().names().get().size()); } @@ -114,7 +112,7 @@ public class WorkerUtilsTest { adminClient.addTopic( false, TEST_TOPIC, - Collections.singletonList(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList())), + List.of(new TopicPartitionInfo(0, broker1, singleReplica, List.of())), null); Map newTopics = new HashMap<>(); @@ -130,8 +128,8 @@ public class WorkerUtilsTest { @Test public void testExistingTopicsMustHaveRequestedNumberOfPartitions() { List tpInfo = new ArrayList<>(); - tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList())); - tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, Collections.emptyList())); + tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, List.of())); + tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, List.of())); adminClient.addTopic( false, TEST_TOPIC, @@ -139,16 +137,16 @@ public class WorkerUtilsTest { null); assertThrows(RuntimeException.class, () -> WorkerUtils.createTopics( - log, adminClient, Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC), false)); + log, adminClient, Map.of(TEST_TOPIC, NEW_TEST_TOPIC), false)); } @Test public void testExistingTopicsNotCreated() throws Throwable { final String existingTopic = "existing-topic"; List tpInfo = new ArrayList<>(); - tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList())); - tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, Collections.emptyList())); - tpInfo.add(new TopicPartitionInfo(2, broker3, singleReplica, Collections.emptyList())); + tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, List.of())); + tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, List.of())); + tpInfo.add(new TopicPartitionInfo(2, broker3, singleReplica, List.of())); adminClient.addTopic( false, existingTopic, @@ -157,11 +155,11 @@ public class WorkerUtilsTest { WorkerUtils.createTopics( log, adminClient, - Collections.singletonMap( + Map.of( existingTopic, new NewTopic(existingTopic, tpInfo.size(), TEST_REPLICATION_FACTOR)), false); - assertEquals(Collections.singleton(existingTopic), adminClient.listTopics().names().get()); + assertEquals(Set.of(existingTopic), adminClient.listTopics().names().get()); } @Test @@ -170,15 +168,15 @@ public class WorkerUtilsTest { assertEquals(0, adminClient.listTopics().names().get().size()); WorkerUtils.createTopics( - log, adminClient, Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC), false); + log, adminClient, Map.of(TEST_TOPIC, NEW_TEST_TOPIC), false); - assertEquals(Collections.singleton(TEST_TOPIC), adminClient.listTopics().names().get()); + assertEquals(Set.of(TEST_TOPIC), adminClient.listTopics().names().get()); assertEquals( new TopicDescription( TEST_TOPIC, false, - Collections.singletonList( - new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList(), Collections.emptyList(), Collections.emptyList()))), - adminClient.describeTopics(Collections.singleton(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() + List.of( + new TopicPartitionInfo(0, broker1, singleReplica, List.of(), List.of(), List.of()))), + adminClient.describeTopics(Set.of(TEST_TOPIC)).topicNameValues().get(TEST_TOPIC).get() ); } @@ -186,8 +184,8 @@ public class WorkerUtilsTest { public void testCreatesOneTopicVerifiesOneTopic() throws Throwable { final String existingTopic = "existing-topic"; List tpInfo = new ArrayList<>(); - tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, Collections.emptyList())); - tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, Collections.emptyList())); + tpInfo.add(new TopicPartitionInfo(0, broker1, singleReplica, List.of())); + tpInfo.add(new TopicPartitionInfo(1, broker2, singleReplica, List.of())); adminClient.addTopic( false, existingTopic, @@ -207,7 +205,7 @@ public class WorkerUtilsTest { @Test public void testCreateNonExistingTopicsWithZeroTopicsDoesNothing() throws Throwable { WorkerUtils.createTopics( - log, adminClient, Collections.emptyMap(), false); + log, adminClient, Map.of(), false); assertEquals(0, adminClient.listTopics().names().get().size()); } @@ -224,8 +222,8 @@ public class WorkerUtilsTest { WorkerUtils.addConfigsToProperties( props, - Collections.singletonMap(ProducerConfig.CLIENT_ID_CONFIG, "test-client"), - Collections.singletonMap(ProducerConfig.LINGER_MS_CONFIG, "1000")); + Map.of(ProducerConfig.CLIENT_ID_CONFIG, "test-client"), + Map.of(ProducerConfig.LINGER_MS_CONFIG, "1000")); assertEquals(resultProps, props); } @@ -242,8 +240,8 @@ public class WorkerUtilsTest { WorkerUtils.addConfigsToProperties( props, - Collections.singletonMap(ProducerConfig.ACKS_CONFIG, "1"), - Collections.singletonMap(ProducerConfig.LINGER_MS_CONFIG, "1000")); + Map.of(ProducerConfig.ACKS_CONFIG, "1"), + Map.of(ProducerConfig.LINGER_MS_CONFIG, "1000")); assertEquals(resultProps, props); } @@ -259,8 +257,8 @@ public class WorkerUtilsTest { WorkerUtils.addConfigsToProperties( props, - Collections.singletonMap(ProducerConfig.ACKS_CONFIG, "1"), - Collections.singletonMap(ProducerConfig.ACKS_CONFIG, "0")); + Map.of(ProducerConfig.ACKS_CONFIG, "1"), + Map.of(ProducerConfig.ACKS_CONFIG, "0")); assertEquals(resultProps, props); } @@ -308,7 +306,7 @@ public class WorkerUtilsTest { for (int i = 0; i < numPartitions; ++i) { Node broker = cluster.get(brokerIndex); tpInfo.add(new TopicPartitionInfo( - i, broker, singleReplica, Collections.emptyList())); + i, broker, singleReplica, List.of())); brokerIndex = (brokerIndex + 1) % cluster.size(); } adminClient.addTopic( @@ -320,14 +318,14 @@ public class WorkerUtilsTest { @Test public void testVerifyTopics() throws Throwable { - Map newTopics = Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC); + Map newTopics = Map.of(TEST_TOPIC, NEW_TEST_TOPIC); WorkerUtils.createTopics(log, adminClient, newTopics, true); adminClient.setFetchesRemainingUntilVisible(TEST_TOPIC, 2); - WorkerUtils.verifyTopics(log, adminClient, Collections.singleton(TEST_TOPIC), - Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC), 3, 1); + WorkerUtils.verifyTopics(log, adminClient, Set.of(TEST_TOPIC), + Map.of(TEST_TOPIC, NEW_TEST_TOPIC), 3, 1); adminClient.setFetchesRemainingUntilVisible(TEST_TOPIC, 100); assertThrows(UnknownTopicOrPartitionException.class, () -> - WorkerUtils.verifyTopics(log, adminClient, Collections.singleton(TEST_TOPIC), - Collections.singletonMap(TEST_TOPIC, NEW_TEST_TOPIC), 2, 1)); + WorkerUtils.verifyTopics(log, adminClient, Set.of(TEST_TOPIC), + Map.of(TEST_TOPIC, NEW_TEST_TOPIC), 2, 1)); } } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java index 313a5db7437..ee5f143cd36 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/coordinator/CoordinatorTest.java @@ -57,7 +57,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.ArrayList; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Optional; @@ -370,7 +369,7 @@ public class CoordinatorTest { private static List> createPartitionLists(String[][] array) { List> list = new ArrayList<>(); for (String[] a : array) { - list.add(Arrays.asList(a)); + list.add(List.of(a)); } return list; } @@ -486,7 +485,7 @@ public class CoordinatorTest { assertEquals(0, coordinatorClient.tasks( new TasksRequest(null, 10, 0, 10, 0, Optional.empty())).tasks().size()); TasksResponse resp1 = coordinatorClient.tasks( - new TasksRequest(Arrays.asList("foo", "baz"), 0, 0, 0, 0, Optional.empty())); + new TasksRequest(List.of("foo", "baz"), 0, 0, 0, 0, Optional.empty())); assertTrue(resp1.tasks().containsKey("foo")); assertFalse(resp1.tasks().containsKey("bar")); assertEquals(1, resp1.tasks().size()); diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java index 595f5f13d42..ba79b7207f3 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/task/SampleTaskSpec.java @@ -20,8 +20,6 @@ package org.apache.kafka.trogdor.task; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import java.util.Collections; -import java.util.HashMap; import java.util.Map; public class SampleTaskSpec extends TaskSpec { @@ -34,8 +32,7 @@ public class SampleTaskSpec extends TaskSpec { @JsonProperty("nodeToExitMs") Map nodeToExitMs, @JsonProperty("error") String error) { super(startMs, durationMs); - this.nodeToExitMs = nodeToExitMs == null ? new HashMap<>() : - Collections.unmodifiableMap(nodeToExitMs); + this.nodeToExitMs = nodeToExitMs == null ? Map.of() : Map.copyOf(nodeToExitMs); this.error = error == null ? "" : error; } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java index e07991d3bf0..7ef1c43e87f 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ConsumeBenchSpecTest.java @@ -22,13 +22,10 @@ import org.apache.kafka.common.TopicPartition; import org.junit.jupiter.api.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -38,7 +35,7 @@ public class ConsumeBenchSpecTest { @Test public void testMaterializeTopicsWithNoPartitions() { - Map> materializedTopics = consumeBenchSpec(Arrays.asList("topic[1-3]", "secondTopic")).materializeTopics(); + Map> materializedTopics = consumeBenchSpec(List.of("topic[1-3]", "secondTopic")).materializeTopics(); Map> expected = new HashMap<>(); expected.put("topic1", new ArrayList<>()); expected.put("topic2", new ArrayList<>()); @@ -50,28 +47,28 @@ public class ConsumeBenchSpecTest { @Test public void testMaterializeTopicsWithSomePartitions() { - Map> materializedTopics = consumeBenchSpec(Arrays.asList("topic[1-3]:[1-5]", "secondTopic", "thirdTopic:1")).materializeTopics(); + Map> materializedTopics = consumeBenchSpec(List.of("topic[1-3]:[1-5]", "secondTopic", "thirdTopic:1")).materializeTopics(); Map> expected = new HashMap<>(); - expected.put("topic1", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic1", (int) i)).collect(Collectors.toList())); - expected.put("topic2", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic2", (int) i)).collect(Collectors.toList())); - expected.put("topic3", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic3", (int) i)).collect(Collectors.toList())); + expected.put("topic1", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic1", (int) i)).toList()); + expected.put("topic2", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic2", (int) i)).toList()); + expected.put("topic3", IntStream.range(1, 6).asLongStream().mapToObj(i -> new TopicPartition("topic3", (int) i)).toList()); expected.put("secondTopic", new ArrayList<>()); - expected.put("thirdTopic", Collections.singletonList(new TopicPartition("thirdTopic", 1))); + expected.put("thirdTopic", List.of(new TopicPartition("thirdTopic", 1))); assertEquals(expected, materializedTopics); } @Test public void testInvalidTopicNameRaisesExceptionInMaterialize() { - for (String invalidName : Arrays.asList("In:valid", "invalid:", ":invalid", "in:valid:1", "invalid:2:2", "invalid::1", "invalid[1-3]:")) { - assertThrows(IllegalArgumentException.class, () -> consumeBenchSpec(Collections.singletonList(invalidName)).materializeTopics()); + for (String invalidName : List.of("In:valid", "invalid:", ":invalid", "in:valid:1", "invalid:2:2", "invalid::1", "invalid[1-3]:")) { + assertThrows(IllegalArgumentException.class, () -> consumeBenchSpec(List.of(invalidName)).materializeTopics()); } } private ConsumeBenchSpec consumeBenchSpec(List activeTopics) { return new ConsumeBenchSpec(0, 0, "node", "localhost", 123, 1234, "cg-1", - Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), 1, + Map.of(), Map.of(), Map.of(), 1, Optional.empty(), activeTopics); } } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java index 852dd3c3b47..0960f8f8955 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ExternalCommandWorkerTest.java @@ -35,7 +35,7 @@ import java.io.File; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import java.util.Arrays; +import java.util.List; import java.util.Optional; import java.util.concurrent.CompletableFuture; @@ -62,7 +62,7 @@ public class ExternalCommandWorkerTest { ExternalCommandSpec spec = new ExternalCommandSpec(0, 30000, "node0", - Arrays.asList(command), + List.of(command), workload, Optional.of(shutdownGracePeriodMs)); return new ExternalCommandWorker(id, spec); diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java index 999cc51ef76..21d1e418070 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/PayloadGeneratorTest.java @@ -22,7 +22,6 @@ import org.junit.jupiter.api.Timeout; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -146,8 +145,8 @@ public class PayloadGeneratorTest { new ConstantPayloadGenerator(4, new byte[0]); RandomComponent constantConfig = new RandomComponent(25, constantGenerator); - List components1 = new ArrayList<>(Arrays.asList(nullConfig, uniformConfig)); - List components2 = new ArrayList<>(Arrays.asList(sequentialConfig, constantConfig)); + List components1 = List.of(nullConfig, uniformConfig); + List components2 = List.of(sequentialConfig, constantConfig); byte[] expected = new byte[4]; PayloadIterator iter = new PayloadIterator( @@ -180,8 +179,8 @@ public class PayloadGeneratorTest { RandomComponent sequentialConfig2 = new RandomComponent(25, sequentialGenerator); RandomComponent nullConfig2 = new RandomComponent(25, nullGenerator); - List components3 = new ArrayList<>(Arrays.asList(sequentialConfig2, uniformConfig2, nullConfig)); - List components4 = new ArrayList<>(Arrays.asList(uniformConfig2, sequentialConfig2, constantConfig, nullConfig2)); + List components3 = List.of(sequentialConfig2, uniformConfig2, nullConfig); + List components4 = List.of(uniformConfig2, sequentialConfig2, constantConfig, nullConfig2); testReproducible(new RandomComponentPayloadGenerator(4, components1)); testReproducible(new RandomComponentPayloadGenerator(123, components2)); @@ -200,12 +199,12 @@ public class PayloadGeneratorTest { new ConstantPayloadGenerator(4, new byte[0]); RandomComponent constantConfig = new RandomComponent(-25, constantGenerator); - List components1 = new ArrayList<>(Arrays.asList(nullConfig, uniformConfig)); - List components2 = new ArrayList<>(Arrays.asList( - nullConfig, constantConfig, uniformConfig, nullConfig, uniformConfig, uniformConfig)); + List components1 = List.of(nullConfig, uniformConfig); + List components2 = List.of( + nullConfig, constantConfig, uniformConfig, nullConfig, uniformConfig, uniformConfig); assertThrows(IllegalArgumentException.class, () -> - new PayloadIterator(new RandomComponentPayloadGenerator(1, new ArrayList<>()))); + new PayloadIterator(new RandomComponentPayloadGenerator(1, List.of()))); assertThrows(IllegalArgumentException.class, () -> new PayloadIterator(new RandomComponentPayloadGenerator(13, components2))); assertThrows(IllegalArgumentException.class, () -> diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpecTest.java index 0dbd57651eb..a4680557248 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/ShareConsumeBenchSpecTest.java @@ -18,10 +18,9 @@ package org.apache.kafka.trogdor.workload; import org.junit.jupiter.api.Test; -import java.util.Arrays; -import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; @@ -32,7 +31,7 @@ class ShareConsumeBenchSpecTest { @Test public void testExpandTopicNames() { - ShareConsumeBenchSpec shareConsumeBenchSpec = shareConsumeBenchSpec(Arrays.asList("foo[1-3]", "bar")); + ShareConsumeBenchSpec shareConsumeBenchSpec = shareConsumeBenchSpec(List.of("foo[1-3]", "bar")); Set expectedNames = new HashSet<>(); expectedNames.add("foo1"); @@ -45,15 +44,15 @@ class ShareConsumeBenchSpecTest { @Test public void testInvalidNameRaisesException() { - for (String invalidName : Arrays.asList("In:valid", "invalid:", ":invalid[]", "in:valid:", "invalid[1-3]:")) { - assertThrows(IllegalArgumentException.class, () -> shareConsumeBenchSpec(Collections.singletonList(invalidName)).expandTopicNames()); + for (String invalidName : List.of("In:valid", "invalid:", ":invalid[]", "in:valid:", "invalid[1-3]:")) { + assertThrows(IllegalArgumentException.class, () -> shareConsumeBenchSpec(List.of(invalidName)).expandTopicNames()); } } @Test public void testDefaultShareGroupName() { ShareConsumeBenchSpec shareConsumeBenchSpec = new ShareConsumeBenchSpec(0, 0, "node", "localhost", - 123, 1234, null, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), 1, + 123, 1234, null, Map.of(), Map.of(), Map.of(), 1, Optional.empty(), List.of("abc")); assertEquals("share", shareConsumeBenchSpec.shareGroup()); } @@ -61,7 +60,7 @@ class ShareConsumeBenchSpecTest { private ShareConsumeBenchSpec shareConsumeBenchSpec(List activeTopics) { return new ShareConsumeBenchSpec(0, 0, "node", "localhost", 123, 1234, "sg-1", - Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), 1, + Map.of(), Map.of(), Map.of(), 1, Optional.empty(), activeTopics); } diff --git a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java index d46585b869c..b63290cc9aa 100644 --- a/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java +++ b/trogdor/src/test/java/org/apache/kafka/trogdor/workload/TopicsSpecTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.trogdor.common.JsonUtil; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -44,8 +43,8 @@ public class TopicsSpecTest { FOO.set("topicA[0-2]", PARTSA); Map> assignmentsB = new HashMap<>(); - assignmentsB.put(0, Arrays.asList(0, 1, 2)); - assignmentsB.put(1, Arrays.asList(2, 3, 4)); + assignmentsB.put(0, List.of(0, 1, 2)); + assignmentsB.put(1, List.of(2, 3, 4)); PARTSB = new PartitionsSpec(0, (short) 0, assignmentsB, null); FOO.set("topicB", PARTSB); } @@ -67,14 +66,14 @@ public class TopicsSpecTest { @Test public void testPartitionNumbers() { List partsANumbers = PARTSA.partitionNumbers(); - assertEquals(Integer.valueOf(0), partsANumbers.get(0)); - assertEquals(Integer.valueOf(1), partsANumbers.get(1)); - assertEquals(Integer.valueOf(2), partsANumbers.get(2)); + assertTrue(partsANumbers.contains(0)); + assertTrue(partsANumbers.contains(1)); + assertTrue(partsANumbers.contains(2)); assertEquals(3, partsANumbers.size()); List partsBNumbers = PARTSB.partitionNumbers(); - assertEquals(Integer.valueOf(0), partsBNumbers.get(0)); - assertEquals(Integer.valueOf(1), partsBNumbers.get(1)); + assertTrue(partsBNumbers.contains(0)); + assertTrue(partsBNumbers.contains(1)); assertEquals(2, partsBNumbers.size()); }