From 0a12eaa80e9f3ff763172f7026b0785fb31c3dbd Mon Sep 17 00:00:00 2001 From: Ken Huang Date: Sat, 6 Sep 2025 01:25:55 +0800 Subject: [PATCH] KAFKA-19112 Unifying LIST-Type Configuration Validation and Default Values (#20334) We add the three main changes in this PR - Disallowing null values for most LIST-type configurations makes sense, since users cannot explicitly set a configuration to null in a properties file. Therefore, only configurations with a default value of null should be allowed to accept null. - Disallowing duplicate values is reasonable, as there are currently no known configurations in Kafka that require specifying the same value multiple times. Allowing duplicates is both rare in practice and potentially confusing to users. - Disallowing empty list, even though many configurations currently accept them. In practice, setting an empty list for several of these configurations can lead to server startup failures or unexpected behavior. Therefore, enforcing non-empty lists helps prevent misconfiguration and improves system robustness. These changes may introduce some backward incompatibility, but this trade-off is justified by the significant improvements in safety, consistency, and overall user experience. Additionally, we introduce two minor adjustments: - Reclassify some STRING-type configurations as LIST-type, particularly those using comma-separated values to represent multiple entries. This change reflects the actual semantics used in Kafka. - Update the default values for some configurations to better align with other configs. These changes will not introduce any compatibility issues. Reviewers: Jun Rao , Chia-Ping Tsai --- .../clients/admin/AdminClientConfig.java | 10 +- .../clients/consumer/ConsumerConfig.java | 26 ++--- .../consumer/ConsumerPartitionAssignor.java | 3 - .../clients/producer/ProducerConfig.java | 22 +++-- .../apache/kafka/common/config/ConfigDef.java | 58 +++++++++-- .../kafka/common/config/SaslConfigs.java | 4 +- .../kafka/common/config/SslConfigs.java | 9 +- .../kafka/common/config/TopicConfig.java | 5 +- .../internals/BrokerSecurityConfigs.java | 4 +- .../oauthbearer/BrokerJwtValidator.java | 5 +- .../security/ssl/DefaultSslEngineFactory.java | 4 +- .../clients/GroupRebalanceConfigTest.java | 11 ++- .../AdminBootstrapAddressesTest.java | 18 ++-- .../clients/consumer/ConsumerConfigTest.java | 13 ++- .../ConsumerPartitionAssignorTest.java | 1 + .../clients/consumer/KafkaConsumerTest.java | 43 ++++++-- .../KafkaShareConsumerMetricsTest.java | 1 + .../consumer/KafkaShareConsumerTest.java | 3 +- .../internals/CommitRequestManagerTest.java | 15 +-- .../internals/FetchRequestManagerTest.java | 1 + .../internals/NetworkClientDelegateTest.java | 2 + .../ShareConsumeRequestManagerTest.java | 5 +- ...reamsGroupHeartbeatRequestManagerTest.java | 3 +- .../TopicMetadataRequestManagerTest.java | 1 + .../clients/producer/KafkaProducerTest.java | 97 +++++++++++++------ .../clients/producer/ProducerConfigTest.java | 14 ++- .../kafka/common/config/ConfigDefTest.java | 55 +++++++++++ .../common/network/SslTransportLayerTest.java | 5 +- .../ssl/DefaultSslEngineFactoryTest.java | 12 ++- .../kafka/test/MockConsumerInterceptor.java | 1 + .../kafka/test/MockProducerInterceptor.java | 1 + .../org/apache/kafka/test/TestSslUtils.java | 7 +- .../connect/mirror/MirrorClientConfig.java | 15 +-- .../connect/mirror/DefaultGroupFilter.java | 2 + .../mirror/MirrorCheckpointConfig.java | 2 + .../mirror/MirrorCheckpointTaskConfig.java | 10 +- .../connect/mirror/MirrorConnectorConfig.java | 2 + .../connect/mirror/MirrorMakerConfig.java | 4 +- .../connect/mirror/MirrorSourceConfig.java | 3 + .../mirror/MirrorSourceTaskConfig.java | 6 +- .../connect/mirror/MirrorMakerConfigTest.java | 14 ++- .../connect/runtime/SinkConnectorConfig.java | 4 +- .../kafka/connect/runtime/WorkerConfig.java | 21 ++-- .../runtime/rest/RestServerConfig.java | 46 ++------- .../connect/runtime/rest/util/SSLUtils.java | 7 +- .../runtime/AbstractWorkerSourceTaskTest.java | 1 + .../connect/runtime/ConnectMetricsTest.java | 8 +- .../runtime/ErrorHandlingTaskTest.java | 1 + .../ExactlyOnceWorkerSourceTaskTest.java | 3 +- .../connect/runtime/MockConnectMetrics.java | 1 + .../SourceTaskOffsetCommitterTest.java | 1 + .../connect/runtime/WorkerSinkTaskTest.java | 1 + .../runtime/WorkerSinkTaskThreadedTest.java | 5 +- .../connect/runtime/WorkerSourceTaskTest.java | 1 + .../kafka/connect/runtime/WorkerTest.java | 3 +- .../distributed/WorkerGroupMemberTest.java | 2 + .../RetryWithToleranceOperatorTest.java | 1 + .../runtime/isolation/MultiVersionTest.java | 1 + .../runtime/isolation/PluginsTest.java | 1 + .../runtime/rest/RestServerConfigTest.java | 2 +- .../standalone/StandaloneConfigTest.java | 1 + .../storage/FileOffsetBackingStoreTest.java | 2 + .../kafka/connect/transforms/DropHeaders.java | 4 +- .../kafka/connect/transforms/MaskField.java | 4 +- .../connect/transforms/ReplaceField.java | 12 ++- .../kafka/connect/transforms/ValueToKey.java | 3 +- .../kafka/metrics/KafkaMetricsConfig.scala | 4 +- .../main/scala/kafka/server/KafkaConfig.scala | 21 ++-- .../main/scala/kafka/utils/CoreUtils.scala | 6 +- .../kafka/server/KRaftClusterTest.scala | 2 +- .../kafka/server/QuorumTestHarness.scala | 6 +- .../scala/unit/kafka/KafkaConfigTest.scala | 41 ++++++-- .../scala/unit/kafka/log/LogConfigTest.scala | 4 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 54 +++++++++++ .../unit/kafka/server/KafkaConfigTest.scala | 23 +++-- docs/configuration.html | 1 + docs/upgrade.html | 29 ++++++ .../group/GroupCoordinatorConfig.java | 6 +- .../kafka/server/config/ServerConfigs.java | 5 +- .../kafka/server/config/ServerLogConfigs.java | 4 +- .../kafka/network/SocketServerConfigs.java | 14 +-- .../server/config/AbstractKafkaConfig.java | 6 +- .../kafka/server/config/KRaftConfigs.java | 4 +- .../server/metrics/ClientMetricsConfigs.java | 14 ++- .../kafka/server/metrics/MetricConfigs.java | 8 +- .../network/SocketServerConfigsTest.java | 22 +++-- .../AddPartitionsToTxnManagerTest.java | 5 +- .../storage/internals/log/LogConfig.java | 9 +- .../storage/internals/log/UnifiedLog.java | 19 +++- .../apache/kafka/streams/StreamsConfig.java | 7 +- .../internals/StateDirectoryTest.java | 6 +- .../TimeOrderedKeyValueBufferTest.java | 2 +- .../processor/MockProcessorContext.java | 2 +- .../processor/api/MockProcessorContext.java | 2 +- .../test/MockProcessorContextAPITest.java | 4 +- .../MockProcessorContextStateStoreTest.java | 2 +- .../kafka/tools/BrokerApiVersionsCommand.java | 2 +- .../kafka/tools/MetadataQuorumCommand.java | 5 +- .../apache/kafka/tools/TopicCommandTest.java | 2 +- 99 files changed, 685 insertions(+), 309 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java index d883ca1450e..471d3916cfb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/admin/AdminClientConfig.java @@ -155,12 +155,14 @@ public class AdminClientConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - "", + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC). define(BOOTSTRAP_CONTROLLERS_CONFIG, Type.LIST, - "", + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_CONTROLLERS_DOC) .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC) @@ -239,6 +241,7 @@ public class AdminClientConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, @@ -284,7 +287,8 @@ public class AdminClientConfig extends AbstractConfig { METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, - List.of(), + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 3fcdf20953c..ee62dc9561b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -37,7 +37,6 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.Utils; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -63,9 +62,9 @@ public class ConsumerConfig extends AbstractConfig { // a list contains all the assignor names that only assign subscribed topics to consumer. Should be updated when new assignor added. // This is to help optimize ConsumerCoordinator#performAssignment method public static final List ASSIGN_FROM_SUBSCRIBED_ASSIGNORS = List.of( - RANGE_ASSIGNOR_NAME, - ROUNDROBIN_ASSIGNOR_NAME, - STICKY_ASSIGNOR_NAME, + RANGE_ASSIGNOR_NAME, + ROUNDROBIN_ASSIGNOR_NAME, + STICKY_ASSIGNOR_NAME, COOPERATIVE_STICKY_ASSIGNOR_NAME ); @@ -406,17 +405,17 @@ public class ConsumerConfig extends AbstractConfig { * A list of configuration keys not supported for CONSUMER protocol. */ private static final List CONSUMER_PROTOCOL_UNSUPPORTED_CONFIGS = List.of( - PARTITION_ASSIGNMENT_STRATEGY_CONFIG, - HEARTBEAT_INTERVAL_MS_CONFIG, + PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + HEARTBEAT_INTERVAL_MS_CONFIG, SESSION_TIMEOUT_MS_CONFIG, SHARE_ACKNOWLEDGEMENT_MODE_CONFIG ); - + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - Collections.emptyList(), - new ConfigDef.NonNullValidator(), + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, @@ -446,7 +445,7 @@ public class ConsumerConfig extends AbstractConfig { .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.LIST, List.of(RangeAssignor.class, CooperativeStickyAssignor.class), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.MEDIUM, PARTITION_ASSIGNMENT_STRATEGY_DOC) .define(METADATA_MAX_AGE_CONFIG, @@ -573,7 +572,7 @@ public class ConsumerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(KEY_DESERIALIZER_CLASS_CONFIG, @@ -614,8 +613,8 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, - Collections.emptyList(), - new ConfigDef.NonNullValidator(), + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(MAX_POLL_RECORDS_CONFIG, @@ -702,6 +701,7 @@ public class ConsumerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java index 20f2551ba6b..45cb505c744 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java @@ -291,9 +291,6 @@ public interface ConsumerPartitionAssignor { // a map to store assignor name -> assignor class name Map assignorNameMap = new HashMap<>(); - if (assignorClasses == null) - return assignors; - for (Object klass : assignorClasses) { // first try to get the class if passed in as a string if (klass instanceof String) { 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 ff7079bcdab..313648497ba 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 @@ -35,7 +35,6 @@ import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -43,6 +42,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; @@ -373,7 +373,12 @@ public class ProducerConfig extends AbstractConfig { private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); static { - CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Collections.emptyList(), new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, ClientDnsLookup.USE_ALL_DNS_IPS.toString(), @@ -462,7 +467,7 @@ public class ProducerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, @@ -501,8 +506,8 @@ public class ProducerConfig extends AbstractConfig { Importance.MEDIUM, PARTITIONER_CLASS_DOC) .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, - Collections.emptyList(), - new ConfigDef.NonNullValidator(), + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, @@ -553,10 +558,11 @@ public class ProducerConfig extends AbstractConfig { atLeast(0), Importance.LOW, CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, - List.of(), - ConfigDef.Importance.LOW, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 970d9cebf72..ee2f8c2cfd9 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -1006,26 +1006,72 @@ public class ConfigDef { public static class ValidList implements Validator { final ValidString validString; + final boolean isEmptyAllowed; + final boolean isNullAllowed; - private ValidList(List validStrings) { + private ValidList(List validStrings, boolean isEmptyAllowed, boolean isNullAllowed) { this.validString = new ValidString(validStrings); + this.isEmptyAllowed = isEmptyAllowed; + this.isNullAllowed = isNullAllowed; + } + + public static ValidList anyNonDuplicateValues(boolean isEmptyAllowed, boolean isNullAllowed) { + return new ValidList(List.of(), isEmptyAllowed, isNullAllowed); } public static ValidList in(String... validStrings) { - return new ValidList(Arrays.asList(validStrings)); + return new ValidList(List.of(validStrings), true, false); + } + + public static ValidList in(boolean isEmptyAllowed, String... validStrings) { + if (!isEmptyAllowed && validStrings.length == 0) { + throw new IllegalArgumentException("At least one valid string must be provided when empty values are not allowed"); + } + return new ValidList(List.of(validStrings), isEmptyAllowed, false); } @Override public void ensureValid(final String name, final Object value) { + if (value == null) { + if (isNullAllowed) + return; + else + throw new ConfigException("Configuration '" + name + "' values must not be null."); + } + @SuppressWarnings("unchecked") - List values = (List) value; - for (String string : values) { - validString.ensureValid(name, string); + List values = (List) value; + if (!isEmptyAllowed && values.isEmpty()) { + String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString(); + throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); + } + + if (Set.copyOf(values).size() != values.size()) { + throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); + } + + validateIndividualValues(name, values); + } + + private void validateIndividualValues(String name, List values) { + boolean hasValidStrings = !validString.validStrings.isEmpty(); + + for (Object value : values) { + if (value instanceof String) { + String string = (String) value; + if (string.isEmpty()) { + throw new ConfigException("Configuration '" + name + "' values must not be empty."); + } + if (hasValidStrings) { + validString.ensureValid(name, value); + } + } } } public String toString() { - return validString.toString(); + return validString + (isEmptyAllowed ? " (empty config allowed)" : " (empty not allowed)") + + (isNullAllowed ? " (null config allowed)" : " (null not allowed)"); } } diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java index b78d96eaac8..01f7ad1f927 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java @@ -19,6 +19,8 @@ package org.apache.kafka.common.config; import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString; import org.apache.kafka.common.config.ConfigDef.Range; +import java.util.List; + public class SaslConfigs { private static final String OAUTHBEARER_NOTE = " Currently applies only to OAUTHBEARER."; @@ -407,7 +409,7 @@ public class SaslConfigs { .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, ConfigDef.Type.LONG, DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, ConfigDef.Type.LONG, DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, ConfigDef.Type.INT, DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC) - .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Type.BOOLEAN, DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_HEADER_URLENCODE_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java index 0b88f44f310..7675f75a9ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java @@ -18,6 +18,7 @@ package org.apache.kafka.common.config; import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; +import java.util.List; import java.util.Set; import javax.net.ssl.KeyManagerFactory; @@ -49,7 +50,9 @@ public class SslConfigs { public static final String SSL_ENABLED_PROTOCOLS_DOC = "The list of protocols enabled for SSL connections. " + "The default is 'TLSv1.2,TLSv1.3'. This means that clients and servers will prefer TLSv1.3 if both support it " + "and fallback to TLSv1.2 otherwise (assuming both support at least TLSv1.2). This default should be fine for most use " - + "cases. Also see the config documentation for ssl.protocol to understand how it can impact the TLS version negotiation behavior."; + + "cases. If this configuration is set to an empty list, Kafka will use the protocols enabled by default in the underlying SSLEngine, " + + "which may include additional protocols depending on the JVM version. " + + "Also see the config documentation for ssl.protocol to understand how it can impact the TLS version negotiation behavior."; public static final String DEFAULT_SSL_ENABLED_PROTOCOLS = "TLSv1.2,TLSv1.3"; public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; @@ -123,8 +126,8 @@ public class SslConfigs { public static void addClientSslSupport(ConfigDef config) { config.define(SslConfigs.SSL_PROTOCOL_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROTOCOL_DOC) .define(SslConfigs.SSL_PROVIDER_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROVIDER_DOC) - .define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, null, ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC) - .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC) .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_PASSWORD_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 958df8abc31..e97c39bc619 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 @@ -159,7 +159,10 @@ public class TopicConfig { "log compaction, which retains the latest value for each key. " + "It is also possible to specify both policies in a comma-separated list (e.g. \"delete,compact\"). " + "In this case, old segments will be discarded per the retention time and size configuration, " + - "while retained segments will be compacted."; + "while retained segments will be compacted. " + + "An empty list means infinite retention - no cleanup policies will be applied and log segments " + + "will be retained indefinitely. Note that with remote storage enabled, local retention limits " + + "(log.local.retention.ms and log.local.retention.bytes) are still applied to local segments."; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = "unclean.leader.election.enable"; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas " + diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index a3a0baf1e39..48f3948ef9d 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java @@ -154,7 +154,7 @@ public class BrokerSecurityConfigs { .define(BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC) .define(SslConfigs.SSL_PROTOCOL_CONFIG, STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, MEDIUM, SslConfigs.SSL_PROTOCOL_DOC) .define(SslConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_PROVIDER_DOC) - .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false), MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC) .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC) @@ -218,6 +218,6 @@ public class BrokerSecurityConfigs { .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, LOW, SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC) - .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, List.of(), LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, STRING, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java index 23ebca74050..c69db033052 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java @@ -106,8 +106,7 @@ public class BrokerJwtValidator implements JwtValidator { @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); - List expectedAudiencesList = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); - Set expectedAudiences = expectedAudiencesList != null ? Set.copyOf(expectedAudiencesList) : null; + Set expectedAudiences = Set.copyOf(cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE)); Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false); String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false); String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); @@ -122,7 +121,7 @@ public class BrokerJwtValidator implements JwtValidator { if (clockSkew != null) jwtConsumerBuilder.setAllowedClockSkewInSeconds(clockSkew); - if (expectedAudiences != null && !expectedAudiences.isEmpty()) + if (!expectedAudiences.isEmpty()) jwtConsumerBuilder.setExpectedAudience(expectedAudiences.toArray(new String[0])); if (expectedIssuer != null) diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java index 0a3d587df90..a5068dc83ab 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java @@ -137,14 +137,14 @@ public class DefaultSslEngineFactory implements SslEngineFactory { SecurityUtils.addConfiguredSecurityProviders(this.configs); List cipherSuitesList = (List) configs.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (cipherSuitesList != null && !cipherSuitesList.isEmpty()) { + if (!cipherSuitesList.isEmpty()) { this.cipherSuites = cipherSuitesList.toArray(new String[0]); } else { this.cipherSuites = null; } List enabledProtocolsList = (List) configs.get(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); - if (enabledProtocolsList != null && !enabledProtocolsList.isEmpty()) { + if (!enabledProtocolsList.isEmpty()) { this.enabledProtocols = enabledProtocolsList.toArray(new String[0]); } else { this.enabledProtocols = null; diff --git a/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java index 7f02204d4c4..41bd0cff42b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java @@ -34,13 +34,14 @@ public class GroupRebalanceConfigTest { GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer" + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); assertTrue(groupRebalanceConfig.rackId.isEmpty()); } - + @ParameterizedTest @EnumSource(value = GroupRebalanceConfig.ProtocolType.class, names = {"CONSUMER", "SHARE"}) void testRackIdIsEmptyIfValueIsEmptyString(GroupRebalanceConfig.ProtocolType protocolType) { @@ -48,7 +49,8 @@ public class GroupRebalanceConfigTest { new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.CLIENT_RACK_CONFIG, "" + ConsumerConfig.CLIENT_RACK_CONFIG, "", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); @@ -62,7 +64,8 @@ public class GroupRebalanceConfigTest { new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.CLIENT_RACK_CONFIG, "rack1" + ConsumerConfig.CLIENT_RACK_CONFIG, "rack1", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java index a61a7bdfda5..0581d672fb8 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java @@ -32,17 +32,13 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; public class AdminBootstrapAddressesTest { - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testNoBootstrapSet(boolean nullValue) { - Map map = new HashMap<>(); - if (nullValue) { - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, null); - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, null); - } else { - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, ""); - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, ""); - } + + @Test + public void testNoBootstrapSet() { + Map map = Map.of( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "", + AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, "" + ); AdminClientConfig config = new AdminClientConfig(map); assertEquals("You must set either bootstrap.servers or bootstrap.controllers", assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index 0e50d5d4698..bed84d67bef 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -59,6 +59,7 @@ public class ConsumerConfigTest { public void setUp() { properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClassName); properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClassName); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); } @Test @@ -147,6 +148,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, ""); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG)); } @@ -157,6 +159,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -167,6 +170,7 @@ public class ConsumerConfigTest { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(saslSslLowerCase, consumerConfig.originals().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); @@ -177,6 +181,7 @@ public class ConsumerConfigTest { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals("classic", consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); assertNull(consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); @@ -191,6 +196,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, remoteAssignorName); configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, protocol); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(protocol, consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); assertEquals(remoteAssignorName, consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); @@ -203,6 +209,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, remoteAssignorName); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException exception = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(exception.getMessage().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG + " cannot be set when " + ConsumerConfig.GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CLASSIC.name())); } @@ -212,6 +219,7 @@ public class ConsumerConfigTest { Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(MetadataRecoveryStrategy.REBOOTSTRAP.name, consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -222,6 +230,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -233,6 +242,7 @@ public class ConsumerConfigTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, protocol); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); if (isValid) { ConsumerConfig config = new ConsumerConfig(configs); assertEquals(protocol, config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); @@ -253,10 +263,11 @@ public class ConsumerConfigTest { ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass, ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass, ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name(), + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092", configName, value ); ConfigException exception = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); - assertEquals(configName + " cannot be set when " + + assertEquals(configName + " cannot be set when " + ConsumerConfig.GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CONSUMER.name(), exception.getMessage()); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java index b4f649de579..c08c7766ec1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java @@ -151,6 +151,7 @@ public class ConsumerPartitionAssignorTest { props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classTypes); props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name()); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ConsumerConfig(props); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index acc2616b8c4..60063e52268 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -716,26 +716,27 @@ public class KafkaConsumerTest { @ParameterizedTest @EnumSource(GroupProtocol.class) public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) { - final int targetInterceptor = 3; + final int targetInterceptor = 1; try { Properties props = new Properties(); props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName() + ", " - + MockConsumerInterceptor.class.getName() + ", " - + MockConsumerInterceptor.class.getName()); + props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, + CloseInterceptor.class.getName() + "," + MockConsumerInterceptor.class.getName()); MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); assertThrows(KafkaException.class, () -> newConsumer( props, new StringDeserializer(), new StringDeserializer())); - assertEquals(3, MockConsumerInterceptor.CONFIG_COUNT.get()); - assertEquals(3, MockConsumerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.CONFIG_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, CloseInterceptor.CLOSE_COUNT.get()); } finally { MockConsumerInterceptor.resetCounters(); + CloseInterceptor.resetCounters(); } } @@ -3115,6 +3116,7 @@ public class KafkaConsumerTest { configs.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, retryBackoffMs); configs.put(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, throwOnStableOffsetNotSupported); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); groupInstanceId.ifPresent(gi -> configs.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, gi)); return new ConsumerConfig(configs); @@ -3798,4 +3800,33 @@ public void testPollIdleRatio(GroupProtocol groupProtocol) { metrics.addMetric(name, (Measurable) (config, now) -> VALUE); } } + + public static class CloseInterceptor implements ConsumerInterceptor { + + public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); + + @Override + public ConsumerRecords onConsume(ConsumerRecords records) { + return null; + } + + @Override + public void onCommit(Map offsets) { + // no-op + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs) { + // no-op + } + + public static void resetCounters() { + CLOSE_COUNT.set(0); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java index ad46bf0887e..a5417c3e00f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java @@ -367,6 +367,7 @@ public class KafkaShareConsumerMetricsTest { configs.put(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG, retryBackoffMaxMs); configs.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, retryBackoffMs); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ShareConsumerConfig(configs); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java index f4b383df67c..1a6d76dbabf 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java @@ -307,6 +307,7 @@ public class KafkaShareConsumerTest { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); configs.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, batchSize); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ShareConsumerConfig(configs); } @@ -414,4 +415,4 @@ public class KafkaShareConsumerTest { .setResponses(new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponseCollection(List.of(topicResponse).iterator())) ); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index 7032c13b285..26d39715d27 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -718,7 +718,7 @@ public class CommitRequestManagerTest { @ParameterizedTest @MethodSource("offsetFetchExceptionSupplier") - public void testOffsetFetchRequestTimeoutRequests(final Errors error, + public void testOffsetFetchRequestTimeoutRequests(final Errors error, final Class expectedExceptionClass) { CommitRequestManager commitRequestManager = create(true, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -917,7 +917,7 @@ public class CommitRequestManagerTest { @ParameterizedTest @MethodSource("offsetCommitExceptionSupplier") public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires( - final Errors error, + final Errors error, final Class expectedExceptionClass) { CommitRequestManager commitRequestManager = create(false, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -1339,7 +1339,7 @@ public class CommitRequestManagerTest { Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, TimeoutException.class), Arguments.of(Errors.REQUEST_TIMED_OUT, TimeoutException.class), Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, TimeoutException.class), - + // Non-retriable errors should result in their specific exceptions Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, GroupAuthorizationException.class), Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, OffsetMetadataTooLarge.class), @@ -1347,7 +1347,7 @@ public class CommitRequestManagerTest { Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED, TopicAuthorizationException.class), Arguments.of(Errors.UNKNOWN_MEMBER_ID, CommitFailedException.class), Arguments.of(Errors.STALE_MEMBER_EPOCH, CommitFailedException.class), - + // Generic errors should result in KafkaException Arguments.of(Errors.UNKNOWN_SERVER_ERROR, KafkaException.class)); } @@ -1364,7 +1364,7 @@ public class CommitRequestManagerTest { Arguments.of(Errors.REQUEST_TIMED_OUT, TimeoutException.class), Arguments.of(Errors.UNSTABLE_OFFSET_COMMIT, TimeoutException.class), Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, TimeoutException.class), - + // Non-retriable errors should result in their specific exceptions Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, GroupAuthorizationException.class), Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, KafkaException.class), @@ -1375,7 +1375,7 @@ public class CommitRequestManagerTest { // Adding STALE_MEMBER_EPOCH as non-retriable here because it is only retried if a new // member epoch is received. Tested separately. Arguments.of(Errors.STALE_MEMBER_EPOCH, StaleMemberEpochException.class), - + // Generic errors should result in KafkaException Arguments.of(Errors.UNKNOWN_SERVER_ERROR, KafkaException.class)); } @@ -1487,7 +1487,7 @@ public class CommitRequestManagerTest { assertEmptyPendingRequests(commitRequestManager); } - + private static void assertEmptyPendingRequests(CommitRequestManager commitRequestManager) { assertTrue(commitRequestManager.pendingRequests.inflightOffsetFetches.isEmpty()); assertTrue(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty()); @@ -1564,6 +1564,7 @@ public class CommitRequestManagerTest { private CommitRequestManager create(final boolean autoCommitEnabled, final long autoCommitInterval) { props.setProperty(AUTO_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(autoCommitInterval)); props.setProperty(ENABLE_AUTO_COMMIT_CONFIG, String.valueOf(autoCommitEnabled)); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); if (autoCommitEnabled) props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 0f83e28d9e4..f806ab65b6b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -4125,6 +4125,7 @@ public class FetchRequestManagerTest { properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(requestTimeoutMs)); properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 24f4aea1c7a..0347423137b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -51,6 +51,7 @@ import java.util.Properties; import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +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.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG; @@ -292,6 +293,7 @@ public class NetworkClientDelegateTest { properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.put(GROUP_ID_CONFIG, GROUP_ID); properties.put(REQUEST_TIMEOUT_MS_CONFIG, REQUEST_TIMEOUT_MS); + properties.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new NetworkClientDelegate(time, new ConsumerConfig(properties), logContext, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index 68da71d7767..a4268b7eca0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -2477,7 +2477,7 @@ public class ShareConsumeRequestManagerTest { // Verify that sensors exist before closing for (String sensorName : sensorNames) { - assertNotNull(metrics.getSensor(sensorName), + assertNotNull(metrics.getSensor(sensorName), "Sensor " + sensorName + " should exist before closing"); } @@ -2486,7 +2486,7 @@ public class ShareConsumeRequestManagerTest { // Verify that all sensors are removed after closing for (String sensorName : sensorNames) { - assertNull(metrics.getSensor(sensorName), + assertNull(metrics.getSensor(sensorName), "Sensor " + sensorName + " should be removed after closing"); } } @@ -2683,6 +2683,7 @@ public class ShareConsumeRequestManagerTest { properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(requestTimeoutMs)); properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate( time, config, logContext, client, metadata, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java index 9839f3b2210..15e23875562 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java @@ -1555,6 +1555,7 @@ class StreamsGroupHeartbeatRequestManagerTest { prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); prop.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, String.valueOf(DEFAULT_MAX_POLL_INTERVAL_MS)); + prop.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ConsumerConfig(prop); } @@ -1623,4 +1624,4 @@ class StreamsGroupHeartbeatRequestManagerTest { .collect(Collectors.toList()); assertEquals(sortedExpected, sortedActual); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java index c977c3ebf3d..f8cc3ee0ccc 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/TopicMetadataRequestManagerTest.java @@ -73,6 +73,7 @@ public class TopicMetadataRequestManagerTest { props.put(ALLOW_AUTO_CREATE_TOPICS_CONFIG, false); props.put(KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); this.topicMetadataRequestManager = spy(new TopicMetadataRequestManager( new LogContext(), time, diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index a52d3c6f0b2..bf27fbaa11f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -235,7 +235,7 @@ public class KafkaProducerTest { ProducerConfig config = new ProducerConfig(props); assertTrue(config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG)); assertTrue(Stream.of("-1", "all").anyMatch(each -> each.equalsIgnoreCase(config.getString(ProducerConfig.ACKS_CONFIG)))); - assertEquals((int) config.getInt(ProducerConfig.RETRIES_CONFIG), Integer.MAX_VALUE); + assertEquals(Integer.MAX_VALUE, (int) config.getInt(ProducerConfig.RETRIES_CONFIG)); assertTrue(config.getString(ProducerConfig.CLIENT_ID_CONFIG).equalsIgnoreCase("producer-" + config.getString(ProducerConfig.TRANSACTIONAL_ID_CONFIG))); } @@ -637,15 +637,15 @@ public class KafkaProducerTest { MockProducerInterceptor.resetCounters(); } } + @Test public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances() { - final int targetInterceptor = 3; + final int targetInterceptor = 1; try { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, org.apache.kafka.test.MockProducerInterceptor.class.getName() + ", " - + org.apache.kafka.test.MockProducerInterceptor.class.getName() + ", " - + org.apache.kafka.test.MockProducerInterceptor.class.getName()); + props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, + CloseInterceptor.class.getName() + "," + MockProducerInterceptor.class.getName()); props.setProperty(MockProducerInterceptor.APPEND_STRING_PROP, "something"); MockProducerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); @@ -654,13 +654,16 @@ public class KafkaProducerTest { new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()) ); - assertEquals(3, MockProducerInterceptor.CONFIG_COUNT.get()); - assertEquals(3, MockProducerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, MockProducerInterceptor.CONFIG_COUNT.get()); + assertEquals(1, MockProducerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, CloseInterceptor.CLOSE_COUNT.get()); } finally { MockProducerInterceptor.resetCounters(); + CloseInterceptor.resetCounters(); } } + @Test public void testPartitionerClose() { try { @@ -1412,12 +1415,12 @@ public class KafkaProducerTest { // Capture flags from the InitProducerIdRequest boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc] - + client.prepareResponse( request -> request instanceof FindCoordinatorRequest && ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE)); - + client.prepareResponse( request -> { if (request instanceof InitProducerIdRequest) { @@ -1429,15 +1432,15 @@ public class KafkaProducerTest { return false; }, initProducerIdResponse(1L, (short) 5, Errors.NONE)); - + try (Producer producer = kafkaProducer(configs, new StringSerializer(), new StringSerializer(), metadata, client, null, time)) { producer.initTransactions(keepPreparedTxn); - + // Verify request flags match expected values - assertEquals(keepPreparedTxn, requestFlags[0], + assertEquals(keepPreparedTxn, requestFlags[0], "keepPreparedTxn flag should match input parameter"); - assertEquals(enable2PC, requestFlags[1], + assertEquals(enable2PC, requestFlags[1], "enable2Pc flag should match producer configuration"); } } @@ -1603,7 +1606,7 @@ public class KafkaProducerTest { ); } } - + @Test public void testCompleteTransactionWithMatchingState() throws Exception { StringSerializer serializer = new StringSerializer(); @@ -1611,37 +1614,37 @@ public class KafkaProducerTest { when(ctx.transactionManager.isPrepared()).thenReturn(true); when(ctx.sender.isRunning()).thenReturn(true); - + // Create prepared states with matching values long producerId = 12345L; short epoch = 5; PreparedTxnState inputState = new PreparedTxnState(producerId, epoch); ProducerIdAndEpoch currentProducerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); - + // Set up the transaction manager to return the prepared state when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentProducerIdAndEpoch); - + // Should trigger commit when states match TransactionalRequestResult commitResult = mock(TransactionalRequestResult.class); when(ctx.transactionManager.beginCommit()).thenReturn(commitResult); - + try (KafkaProducer producer = ctx.newKafkaProducer()) { // Call completeTransaction with the matching state producer.completeTransaction(inputState); - + // Verify methods called in order verify(ctx.transactionManager).isPrepared(); verify(ctx.transactionManager).preparedTransactionState(); verify(ctx.transactionManager).beginCommit(); - + // Verify abort was never called verify(ctx.transactionManager, never()).beginAbort(); - + // Verify sender was woken up verify(ctx.sender).wakeup(); } } - + @Test public void testCompleteTransactionWithNonMatchingState() throws Exception { StringSerializer serializer = new StringSerializer(); @@ -1649,37 +1652,37 @@ public class KafkaProducerTest { when(ctx.transactionManager.isPrepared()).thenReturn(true); when(ctx.sender.isRunning()).thenReturn(true); - + // Create txn prepared states with different values long producerId = 12345L; short epoch = 5; PreparedTxnState inputState = new PreparedTxnState(producerId + 1, epoch); ProducerIdAndEpoch currentProducerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); - + // Set up the transaction manager to return the prepared state when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentProducerIdAndEpoch); - + // Should trigger abort when states don't match TransactionalRequestResult abortResult = mock(TransactionalRequestResult.class); when(ctx.transactionManager.beginAbort()).thenReturn(abortResult); - + try (KafkaProducer producer = ctx.newKafkaProducer()) { // Call completeTransaction with the non-matching state producer.completeTransaction(inputState); - + // Verify methods called in order verify(ctx.transactionManager).isPrepared(); verify(ctx.transactionManager).preparedTransactionState(); verify(ctx.transactionManager).beginAbort(); - + // Verify commit was never called verify(ctx.transactionManager, never()).beginCommit(); - + // Verify sender was woken up verify(ctx.sender).wakeup(); } } - + @Test public void testClusterAuthorizationFailure() throws Exception { int maxBlockMs = 500; @@ -3204,4 +3207,38 @@ public class KafkaProducerTest { metrics.addMetric(name, (Measurable) (config, now) -> VALUE); } } + + public static class CloseInterceptor implements ProducerInterceptor { + + public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); + + @Override + public ProducerRecord onSend(ProducerRecord record) { + return null; + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception) { + ProducerInterceptor.super.onAcknowledgement(metadata, exception); + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + ProducerInterceptor.super.onAcknowledgement(metadata, exception, headers); + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs) { + // no-op + } + + public static void resetCounters() { + CLOSE_COUNT.set(0); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java index 33f069c8523..5fd9ab727e0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/ProducerConfigTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.MetadataRecoveryStrategy; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -92,6 +93,7 @@ public class ProducerConfigTest { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(ProducerConfig.COMPRESSION_TYPE_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); } @@ -101,6 +103,7 @@ public class ProducerConfigTest { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -110,6 +113,7 @@ public class ProducerConfigTest { Map configs = new HashMap<>(); configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ProducerConfig producerConfig = new ProducerConfig(configs); assertEquals(MetadataRecoveryStrategy.REBOOTSTRAP.name, producerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -120,6 +124,7 @@ public class ProducerConfigTest { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -131,6 +136,7 @@ public class ProducerConfigTest { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ProducerConfig producerConfig = new ProducerConfig(configs); assertEquals(saslSslLowerCase, producerConfig.originals().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -142,6 +148,7 @@ public class ProducerConfigTest { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, inFlightConnection); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException configException = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertEquals("To use the idempotent producer, " + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " must be set to at most 5. Current value is " + inFlightConnection + ".", configException.getMessage()); @@ -159,15 +166,16 @@ public class ProducerConfigTest { configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); - + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG)); assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG)); - + // Verify that setting one but not the other is valid configs.remove(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); assertDoesNotThrow(() -> new ProducerConfig(configs)); - + configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, false); assertDoesNotThrow(() -> new ProducerConfig(configs)); diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 46791c18471..c6c2390b07c 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -758,4 +758,59 @@ public class ConfigDefTest { assertEquals("List containing maximum of 5 elements", ListSize.atMostOfSize(5).toString()); } + @Test + public void testListValidatorAnyNonDuplicateValues() { + ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyNonDuplicateValues(true, true); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "b", "c"))); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); + ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); + + ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyNonDuplicateValues(false, true); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "b", "c"))); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", null)); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception3.getMessage()); + ConfigException exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception5.getMessage()); + + ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyList = ConfigDef.ValidList.anyNonDuplicateValues(true, false); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "b", "c"))); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of())); + ConfigException exception6 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception6.getMessage()); + ConfigException exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception7.getMessage()); + ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception8.getMessage()); + } + + @Test + public void testListValidatorIn() { + ConfigDef.ValidList allowEmptyValidator = ConfigDef.ValidList.in(true, "a", "b", "c"); + assertDoesNotThrow(() -> allowEmptyValidator.ensureValid("test.config", List.of("a", "b"))); + assertDoesNotThrow(() -> allowEmptyValidator.ensureValid("test.config", List.of())); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception1.getMessage()); + ConfigException exception2 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", List.of("d"))); + assertEquals("Invalid value d for configuration test.config: String must be one of: a, b, c", exception2.getMessage()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception3.getMessage()); + + ConfigDef.ValidList notAllowEmptyValidator = ConfigDef.ValidList.in(false, "a", "b", "c"); + assertDoesNotThrow(() -> notAllowEmptyValidator.ensureValid("test.config", List.of("a", "b"))); + ConfigException exception4 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: [a, b, c]", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception5.getMessage()); + ConfigException exception6 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of("d"))); + assertEquals("Invalid value d for configuration test.config: String must be one of: a, b, c", exception6.getMessage()); + ConfigException exception7 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception7.getMessage()); + } } diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index d75028b1c80..72f130ca4e3 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -110,7 +110,8 @@ public class SslTransportLayerTest { this.useInlinePem = useInlinePem; sslConfigOverrides = new HashMap<>(); sslConfigOverrides.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocol); - sslConfigOverrides.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList(tlsProtocol)); + sslConfigOverrides.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(tlsProtocol)); + sslConfigOverrides.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); init(); } @@ -607,7 +608,7 @@ public class SslTransportLayerTest { public void testTlsDefaults(Args args) throws Exception { args.sslServerConfigs = args.serverCertStores.getTrustingConfig(args.clientCertStores); args.sslClientConfigs = args.clientCertStores.getTrustingConfig(args.serverCertStores); - + args.sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, args.sslServerConfigs.get(SslConfigs.SSL_PROTOCOL_CONFIG)); assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, args.sslClientConfigs.get(SslConfigs.SSL_PROTOCOL_CONFIG)); diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java index 232d4d7327b..ed76495fa03 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java @@ -202,6 +202,8 @@ public class DefaultSslEngineFactoryTest { public void setUp() { factory = sslEngineFactory(); configs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2"); + configs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); + configs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of()); } protected DefaultSslEngineFactory sslEngineFactory() { @@ -216,7 +218,7 @@ public class DefaultSslEngineFactoryTest { KeyStore trustStore = factory.truststore(); List aliases = Collections.list(trustStore.aliases()); - assertEquals(Collections.singletonList("kafka0"), aliases); + assertEquals(List.of("kafka0"), aliases); assertNotNull(trustStore.getCertificate("kafka0"), "Certificate not loaded"); assertNull(trustStore.getKey("kafka0", null), "Unexpected private key"); } @@ -270,7 +272,7 @@ public class DefaultSslEngineFactoryTest { KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not loaded"); assertNotNull(keyStore.getKey("kafka", keyPassword == null ? null : keyPassword.value().toCharArray()), "Private key not loaded"); @@ -284,7 +286,7 @@ public class DefaultSslEngineFactoryTest { KeyStore trustStore = factory.truststore(); List aliases = Collections.list(trustStore.aliases()); - assertEquals(Collections.singletonList("kafka0"), aliases); + assertEquals(List.of("kafka0"), aliases); assertNotNull(trustStore.getCertificate("kafka0"), "Certificate not found"); assertNull(trustStore.getKey("kafka0", null), "Unexpected private key"); } @@ -299,7 +301,7 @@ public class DefaultSslEngineFactoryTest { KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not loaded"); assertNotNull(keyStore.getKey("kafka", null), "Private key not loaded"); } @@ -314,7 +316,7 @@ public class DefaultSslEngineFactoryTest { KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not found"); assertNotNull(keyStore.getKey("kafka", KEY_PASSWORD.value().toCharArray()), "Private key not found"); } diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java index a1d676e15a3..715486c1ae3 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -118,6 +118,7 @@ public class MockConsumerInterceptor implements ClusterResourceListener, Consume CONFIG_COUNT.set(0); THROW_CONFIG_EXCEPTION.set(0); CLUSTER_META.set(null); + THROW_ON_CONFIG_EXCEPTION_THRESHOLD.set(0); CLUSTER_ID_BEFORE_ON_CONSUME.set(NO_CLUSTER_ID); } diff --git a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java index acc69ab44e3..9e69f57c96f 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java @@ -110,6 +110,7 @@ public class MockProducerInterceptor implements ClusterResourceListener, Produce ON_SUCCESS_COUNT.set(0); ON_ERROR_COUNT.set(0); ON_ERROR_WITH_METADATA_COUNT.set(0); + THROW_ON_CONFIG_EXCEPTION_THRESHOLD.set(0); CLUSTER_META.set(null); CLUSTER_ID_BEFORE_ON_ACKNOWLEDGEMENT.set(NO_CLUSTER_ID); } diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index b89aade3686..47a26aa697d 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -84,7 +84,6 @@ import java.security.cert.Certificate; import java.security.cert.CertificateException; import java.security.cert.X509Certificate; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.Enumeration; import java.util.HashMap; @@ -211,6 +210,7 @@ public class TestSslUtils { sslConfigs.put(SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, keyManagerAlgorithm); sslConfigs.put(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, trustManagerAlgorithm); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); @@ -686,6 +686,7 @@ public class TestSslUtils { sslConfigs.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, trustStorePassword); sslConfigs.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); sslConfigs.put(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, TrustManagerFactory.getDefaultAlgorithm()); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); @@ -701,7 +702,8 @@ public class TestSslUtils { Map sslConfigs = new HashMap<>(); sslConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocol); - sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList(tlsProtocol)); + sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(tlsProtocol)); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); if (connectionMode != ConnectionMode.CLIENT || useClientCert) { KeyPair keyPair = generateKeyPair(algorithm); @@ -838,6 +840,7 @@ public class TestSslUtils { List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, enabledProtocols); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); return sslConfigs; } diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java index 53a4f9f5f05..cb42f5fe654 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java @@ -32,6 +32,7 @@ import java.util.HashMap; import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; /** * Configuration required for {@link MirrorClient} to talk to a given target cluster. @@ -105,7 +106,7 @@ public class MirrorClientConfig extends AbstractConfig { public Map producerConfig() { return clientConfig(PRODUCER_CLIENT_PREFIX); } - + private Map clientConfig(String prefix) { Map props = new HashMap<>(valuesWithPrefixOverride(prefix)); props.keySet().retainAll(CLIENT_CONFIG_DEF.names()); @@ -117,7 +118,8 @@ public class MirrorClientConfig extends AbstractConfig { static final ConfigDef CLIENT_CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - null, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) // security support @@ -129,13 +131,14 @@ public class MirrorClientConfig extends AbstractConfig { CommonClientConfigs.SECURITY_PROTOCOL_DOC) .withClientSslSupport() .withClientSaslSupport(); - + static final ConfigDef CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, - Type.STRING, - null, + Type.LIST, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( REPLICATION_POLICY_CLASS, ConfigDef.Type.CLASS, diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java index f88ed4e7046..ded82a8571c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/DefaultGroupFilter.java @@ -65,11 +65,13 @@ public class DefaultGroupFilter implements GroupFilter { .define(GROUPS_INCLUDE_CONFIG, Type.LIST, GROUPS_INCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, GROUPS_INCLUDE_DOC) .define(GROUPS_EXCLUDE_CONFIG, Type.LIST, GROUPS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, GROUPS_EXCLUDE_DOC); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java index 3fb2859d2dd..b7625da619d 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java @@ -194,12 +194,14 @@ public class MirrorCheckpointConfig extends MirrorConnectorConfig { GROUPS, ConfigDef.Type.LIST, GROUPS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, GROUPS_DOC) .define( GROUPS_EXCLUDE, ConfigDef.Type.LIST, GROUPS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, GROUPS_EXCLUDE_DOC) .define( diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java index 0617a4728c2..3d2cfda6dcc 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java @@ -19,7 +19,6 @@ package org.apache.kafka.connect.mirror; import org.apache.kafka.common.config.ConfigDef; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; @@ -32,11 +31,7 @@ public class MirrorCheckpointTaskConfig extends MirrorCheckpointConfig { } Set taskConsumerGroups() { - List fields = getList(TASK_CONSUMER_GROUPS); - if (fields == null || fields.isEmpty()) { - return Set.of(); - } - return new HashSet<>(fields); + return new HashSet<>(getList(TASK_CONSUMER_GROUPS)); } MirrorCheckpointMetrics metrics() { @@ -54,7 +49,8 @@ public class MirrorCheckpointTaskConfig extends MirrorCheckpointConfig { .define( TASK_CONSUMER_GROUPS, ConfigDef.Type.LIST, - null, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.LOW, TASK_CONSUMER_GROUPS_DOC) .define(TASK_INDEX, diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java index 849f40c1fce..9baf7c1f35c 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java @@ -311,6 +311,7 @@ public abstract class MirrorConnectorConfig extends AbstractConfig { CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, ConfigDef.Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define( @@ -323,6 +324,7 @@ public abstract class MirrorConnectorConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) .withClientSslSupport() diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 7bcb3b5f742..33fc2641a39 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -279,9 +279,9 @@ public final class MirrorMakerConfig extends AbstractConfig { private static ConfigDef config() { ConfigDef result = new ConfigDef() - .define(CLUSTERS_CONFIG, Type.LIST, Importance.HIGH, CLUSTERS_DOC) + .define(CLUSTERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, CLUSTERS_DOC) .define(ENABLE_INTERNAL_REST_CONFIG, Type.BOOLEAN, false, Importance.HIGH, ENABLE_INTERNAL_REST_DOC) - .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) // security support .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java index dc0da538233..2ec663ad2fc 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java @@ -209,18 +209,21 @@ public class MirrorSourceConfig extends MirrorConnectorConfig { TOPICS, ConfigDef.Type.LIST, TOPICS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC) .define( TOPICS_EXCLUDE, ConfigDef.Type.LIST, TOPICS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_EXCLUDE_DOC) .define( CONFIG_PROPERTIES_EXCLUDE, ConfigDef.Type.LIST, CONFIG_PROPERTIES_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, CONFIG_PROPERTIES_EXCLUDE_DOC) .define( diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java index 3d99d12cdd5..aa5d300c00a 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java @@ -34,9 +34,6 @@ public class MirrorSourceTaskConfig extends MirrorSourceConfig { Set taskTopicPartitions() { List fields = getList(TASK_TOPIC_PARTITIONS); - if (fields == null || fields.isEmpty()) { - return Set.of(); - } return fields.stream() .map(MirrorUtils::decodeTopicPartition) .collect(Collectors.toSet()); @@ -57,7 +54,8 @@ public class MirrorSourceTaskConfig extends MirrorSourceConfig { .define( TASK_TOPIC_PARTITIONS, ConfigDef.Type.LIST, - null, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.LOW, TASK_TOPIC_PARTITIONS_DOC) .define(TASK_INDEX, diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java index 7dfdf460c44..ddd22b0b8ad 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -104,7 +105,7 @@ public class MirrorMakerConfigTest { "replication.policy.separator is picked up in MirrorClientConfig"); assertEquals("b__topic1", aClientConfig.replicationPolicy().formatRemoteTopic("b", "topic1"), "replication.policy.separator is honored"); - assertEquals(clusterABootstrap, aClientConfig.adminConfig().get("bootstrap.servers"), + assertEquals(Arrays.stream(clusterABootstrap.split(",")).map(String::trim).toList(), aClientConfig.adminConfig().get("bootstrap.servers"), "client configs include bootstrap.servers"); try (ForwardingAdmin forwardingAdmin = aClientConfig.forwardingAdmin(aClientConfig.adminConfig())) { assertEquals(ForwardingAdmin.class.getName(), forwardingAdmin.getClass().getName(), @@ -178,7 +179,7 @@ public class MirrorMakerConfigTest { SourceAndTarget sourceAndTarget = new SourceAndTarget("source", "target"); Map connectorProps = mirrorConfig.connectorBaseConfig(sourceAndTarget, MirrorSourceConnector.class); - DefaultTopicFilter.TopicFilterConfig filterConfig = + DefaultTopicFilter.TopicFilterConfig filterConfig = new DefaultTopicFilter.TopicFilterConfig(connectorProps); assertEquals(List.of("topic1", "topic2"), filterConfig.getList("topics"), "source->target.topics should be passed through to TopicFilters."); @@ -316,7 +317,10 @@ public class MirrorMakerConfigTest { @Test public void testClientInvalidSecurityProtocol() { ConfigException ce = assertThrows(ConfigException.class, - () -> new MirrorClientConfig(makeProps(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"))); + () -> new MirrorClientConfig(makeProps( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc", + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + ))); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -324,7 +328,9 @@ public class MirrorMakerConfigTest { public void testCaseInsensitiveSecurityProtocol() { final String saslSslLowerCase = SecurityProtocol.SASL_SSL.name.toLowerCase(Locale.ROOT); final MirrorClientConfig config = new MirrorClientConfig(makeProps( - CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase)); + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase, + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + )); assertEquals(saslSslLowerCase, config.originalsStrings().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java index b52b73810fd..43da717a78c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SinkConnectorConfig.java @@ -41,7 +41,7 @@ public class SinkConnectorConfig extends ConnectorConfig { public static final String TOPICS_CONFIG = SinkTask.TOPICS_CONFIG; private static final String TOPICS_DOC = "List of topics to consume, separated by commas"; - public static final String TOPICS_DEFAULT = ""; + public static final List TOPICS_DEFAULT = List.of(); private static final String TOPICS_DISPLAY = "Topics"; public static final String TOPICS_REGEX_CONFIG = SinkTask.TOPICS_REGEX_CONFIG; @@ -74,7 +74,7 @@ public class SinkConnectorConfig extends ConnectorConfig { private static ConfigDef configDef(ConfigDef baseConfigs) { return baseConfigs - .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) + .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) .define(TOPICS_REGEX_CONFIG, ConfigDef.Type.STRING, TOPICS_REGEX_DEFAULT, new RegexValidator(), ConfigDef.Importance.HIGH, TOPICS_REGEX_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_REGEX_DISPLAY) .define(DLQ_TOPIC_NAME_CONFIG, ConfigDef.Type.STRING, DLQ_TOPIC_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_NAME_DOC, ERROR_GROUP, 6, ConfigDef.Width.MEDIUM, DLQ_TOPIC_DISPLAY) .define(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, ConfigDef.Type.SHORT, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DOC, ERROR_GROUP, 7, ConfigDef.Width.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DISPLAY) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index bec7d2085ec..8d953d7ded3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -58,13 +58,12 @@ public class WorkerConfig extends AbstractConfig { private static final Logger log = LoggerFactory.getLogger(WorkerConfig.class); public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - public static final String BOOTSTRAP_SERVERS_DOC = + public static final String BOOTSTRAP_SERVERS_DOC = "A list of host/port pairs used to establish the initial connection to the Kafka cluster. " + "Clients use this list to bootstrap and discover the full set of Kafka brokers. " + "While the order of servers in the list does not matter, we recommend including more than one server to ensure resilience if any servers are down. " + "This list does not need to contain the entire set of brokers, as Kafka clients automatically manage and update connections to the cluster efficiently. " + "This list must be in the form host1:port1,host2:port2,...."; - public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG; public static final String CLIENT_DNS_LOOKUP_DOC = CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC; @@ -136,7 +135,7 @@ public class WorkerConfig extends AbstractConfig { + "plugins and their dependencies\n" + "Note: symlinks will be followed to discover dependencies or plugins.\n" + "Examples: plugin.path=/usr/local/share/java,/usr/local/share/kafka/plugins," - + "/opt/connectors\n" + + "/opt/connectors\n" + "Do not use config provider variables in this property, since the raw path is used " + "by the worker's scanner before config providers are initialized and used to " + "replace variables."; @@ -198,7 +197,8 @@ public class WorkerConfig extends AbstractConfig { */ protected static ConfigDef baseConfigDef() { ConfigDef result = new ConfigDef() - .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT, + .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, @@ -225,6 +225,7 @@ public class WorkerConfig extends AbstractConfig { .define(PLUGIN_PATH_CONFIG, Type.LIST, null, + ConfigDef.ValidList.anyNonDuplicateValues(false, true), Importance.LOW, PLUGIN_PATH_DOC) .define(PLUGIN_DISCOVERY_CONFIG, @@ -245,15 +246,19 @@ public class WorkerConfig extends AbstractConfig { Importance.LOW, CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, - JmxReporter.class.getName(), Importance.LOW, + JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS, HEADER_CONVERTER_CLASS_DEFAULT, Importance.LOW, HEADER_CONVERTER_CLASS_DOC) .define(HEADER_CONVERTER_VERSION, Type.STRING, HEADER_CONVERTER_VERSION_DEFAULT, Importance.LOW, HEADER_CONVERTER_VERSION_DOC) - .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, + .define(CONFIG_PROVIDERS_CONFIG, + Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) .define(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, Type.STRING, CONNECTOR_CLIENT_POLICY_CLASS_DEFAULT, Importance.MEDIUM, CONNECTOR_CLIENT_POLICY_CLASS_DOC) @@ -337,8 +342,8 @@ public class WorkerConfig extends AbstractConfig { if (!Objects.equals(rawPluginPath, transformedPluginPath)) { log.warn( "Variables cannot be used in the 'plugin.path' property, since the property is " - + "used by plugin scanning before the config providers that replace the " - + "variables are initialized. The raw value '{}' was used for plugin scanning, as " + + "used by plugin scanning before the config providers that replace the " + + "variables are initialized. The raw value '{}' was used for plugin scanning, as " + "opposed to the transformed value '{}', and this may cause unexpected results.", rawPluginPath, transformedPluginPath diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java index 3c4980675a0..1a08a7eb123 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServerConfig.java @@ -85,7 +85,8 @@ public abstract class RestServerConfig extends AbstractConfig { private static final String ADMIN_LISTENERS_DOC = "List of comma-separated URIs the Admin REST API will listen on." + " The supported protocols are HTTP and HTTPS." + " An empty or blank string will disable this feature." + - " The default behavior is to use the regular listener (specified by the 'listeners' property)."; + " The default behavior is to use the regular listener (specified by the 'listeners' property)." + + " A comma-separated list of valid URLs, e.g., http://localhost:8080,https://localhost:8443."; public static final String ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX = "admin.listeners.https."; public static final String REST_EXTENSION_CLASSES_CONFIG = "rest.extension.classes"; @@ -139,15 +140,15 @@ public abstract class RestServerConfig extends AbstractConfig { public static void addPublicConfig(ConfigDef configDef) { addInternalConfig(configDef); configDef - .define( - REST_EXTENSION_CLASSES_CONFIG, + .define(REST_EXTENSION_CLASSES_CONFIG, ConfigDef.Type.LIST, - "", - ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC - ).define(ADMIN_LISTENERS_CONFIG, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC) + .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - new AdminListenersValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, true), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } @@ -328,37 +329,6 @@ public abstract class RestServerConfig extends AbstractConfig { } } - private static class AdminListenersValidator implements ConfigDef.Validator { - @Override - public void ensureValid(String name, Object value) { - if (value == null) { - return; - } - - if (!(value instanceof List items)) { - throw new ConfigException("Invalid value type for admin.listeners (expected list)."); - } - - if (items.isEmpty()) { - return; - } - - for (Object item : items) { - if (!(item instanceof String)) { - throw new ConfigException("Invalid type for admin.listeners (expected String)."); - } - if (Utils.isBlank((String) item)) { - throw new ConfigException("Empty URL found when parsing admin.listeners list."); - } - } - } - - @Override - public String toString() { - return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443."; - } - } - private static class ResponseHttpHeadersValidator implements ConfigDef.Validator { @Override public void ensureValid(String name, Object value) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java index 33e187130d4..83a175e8d5f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java @@ -123,7 +123,9 @@ public class SSLUtils { @SuppressWarnings("unchecked") protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl, Map sslConfigValues) { List sslEnabledProtocols = (List) getOrDefault(sslConfigValues, SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(COMMA_WITH_WHITESPACE.split(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS))); - ssl.setIncludeProtocols(sslEnabledProtocols.toArray(new String[0])); + + if (!sslEnabledProtocols.isEmpty()) + ssl.setIncludeProtocols(sslEnabledProtocols.toArray(new String[0])); String sslProvider = (String) sslConfigValues.get(SslConfigs.SSL_PROVIDER_CONFIG); if (sslProvider != null) @@ -132,7 +134,8 @@ public class SSLUtils { ssl.setProtocol((String) getOrDefault(sslConfigValues, SslConfigs.SSL_PROTOCOL_CONFIG, SslConfigs.DEFAULT_SSL_PROTOCOL)); List sslCipherSuites = (List) sslConfigValues.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (sslCipherSuites != null) + + if (!sslCipherSuites.isEmpty()) ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[0])); ssl.setKeyManagerFactoryAlgorithm((String) getOrDefault(sslConfigValues, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM)); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java index 90eac8b7982..d5b15dde76f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java @@ -169,6 +169,7 @@ public class AbstractWorkerSourceTaskTest { private Map workerProps() { Map props = new HashMap<>(); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java index b4733157587..8ba0316e20c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java @@ -62,15 +62,17 @@ public class ConnectMetricsTest { private static final Map DEFAULT_WORKER_CONFIG = Map.of( WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter", - WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); + WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter", + WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + ); private static final ConnectorTaskId CONNECTOR_TASK_ID = new ConnectorTaskId("connector", 0); private static final LinkedHashMap TAGS = new LinkedHashMap<>(); private ConnectMetrics metrics; - + static { TAGS.put("t1", "v1"); } - + @BeforeEach public void setUp() { metrics = new ConnectMetrics("worker1", new WorkerConfig(WorkerConfig.baseConfigDef(), DEFAULT_WORKER_CONFIG), new MockTime(), "cluster-1"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index 28a9966bcf2..a8e001544b3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -184,6 +184,7 @@ public class ErrorHandlingTaskTest { workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); workerConfig = new StandaloneConfig(workerProps); sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java index 69e72dab315..4067f5aa59d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java @@ -229,6 +229,7 @@ public class ExactlyOnceWorkerSourceTaskTest { private Map workerProps() { Map props = new HashMap<>(); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); @@ -635,7 +636,7 @@ public class ExactlyOnceWorkerSourceTaskTest { time.sleep(commitInterval * 2); awaitPolls(2); - assertEquals(2, flushCount(), + assertEquals(2, flushCount(), "Two flushes should have taken place after offset commit interval has elapsed again"); awaitShutdown(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java index c5f9f8314d9..b6548651418 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java @@ -49,6 +49,7 @@ public class MockConnectMetrics extends ConnectMetrics { DEFAULT_WORKER_CONFIG.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); DEFAULT_WORKER_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); DEFAULT_WORKER_CONFIG.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + DEFAULT_WORKER_CONFIG.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); } public MockConnectMetrics() { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java index 1187785a726..0b1f4efc850 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java @@ -72,6 +72,7 @@ public class SourceTaskOffsetCommitterTest { @BeforeEach public void setup() { Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index ab5cfc029ad..ce052dd2439 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -179,6 +179,7 @@ public class WorkerSinkTaskTest { public void setUp() { time = new MockTime(); Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 3f9cf168fe9..729b5f0436c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -171,6 +171,7 @@ public class WorkerSinkTaskThreadedTest { workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); WorkerConfig workerConfig = new StandaloneConfig(workerProps); Plugin keyConverterPlugin = metrics.wrap(keyConverter, taskId, true); Plugin valueConverterPlugin = metrics.wrap(valueConverter, taskId, false); @@ -648,7 +649,7 @@ public class WorkerSinkTaskThreadedTest { @Override public Object answer(InvocationOnMock invocation) { ExpectOffsetCommitCommand commitCommand = commands[index++]; - // All assigned partitions will have offsets committed, but we've only processed messages/updated + // All assigned partitions will have offsets committed, but we've only processed messages/updated // offsets for one final Map offsetsToCommit = offsetsToCommitFn.apply(commitCommand.expectedMessages); @@ -661,7 +662,7 @@ public class WorkerSinkTaskThreadedTest { } }).when(sinkTask).preCommit(anyMap()); } - + private void expectOffsetCommit(ExpectOffsetCommitCommand... commands) { doAnswer(new Answer<>() { int index = 0; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 608b4e642f0..4fca3f8ac61 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -202,6 +202,7 @@ public class WorkerSourceTaskTest { props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("offset.storage.file.filename", "/tmp/connect.offsets"); props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return props; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index ce048836df8..e29eeebe88d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -249,6 +249,7 @@ public class WorkerTest { .strictness(Strictness.STRICT_STUBS) .startMocking(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); @@ -2863,7 +2864,7 @@ public class WorkerTest { ArgumentCaptor failureCaptor = ArgumentCaptor.forClass(Throwable.class); verify(taskStatusListener, times(1)).onFailure(eq(TASK_ID), failureCaptor.capture()); - assertInstanceOf(TooManyTasksException.class, failureCaptor.getValue(), + assertInstanceOf(TooManyTasksException.class, failureCaptor.getValue(), "Expected task start exception to be TooManyTasksException, but was " + failureCaptor.getValue().getClass() + " instead"); tasksMaxExceededMessage = failureCaptor.getValue().getMessage(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java index fad39d84129..4886431869c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java @@ -59,6 +59,7 @@ public class WorkerGroupMemberTest { public void testMetrics() throws Exception { WorkerGroupMember member; Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); @@ -102,6 +103,7 @@ public class WorkerGroupMemberTest { public void testDisableJmxReporter() { WorkerGroupMember member; Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("group.id", "group-1"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java index a56a63ae4db..f9e47afd0b3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java @@ -88,6 +88,7 @@ public class RetryWithToleranceOperatorTest { put(CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.INFO.toString()); // define required properties + put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); }}; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java index 8b074b38755..c4a60df57e1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java @@ -266,6 +266,7 @@ public class MultiVersionTest { converterConfig.put(WorkerConfig.VALUE_CONVERTER_VERSION, "2.3.0"); converterConfig.put(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, VersionedPluginBuilder.VersionedTestPlugin.HEADER_CONVERTER.className()); converterConfig.put(WorkerConfig.HEADER_CONVERTER_VERSION, "4.3.0"); + converterConfig.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); AbstractConfig config; try (LoaderSwap swap = plugins.safeLoaderSwapper().apply(plugins.delegatingLoader())) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java index 2c2b384b7b5..9492f9f7ea2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java @@ -92,6 +92,7 @@ public class PluginsTest { pluginProps.put(WorkerConfig.PLUGIN_PATH_CONFIG, TestPlugins.pluginPathJoined()); plugins = new Plugins(pluginProps); props = new HashMap<>(pluginProps); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); props.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); props.put("key.converter." + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "true"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java index c9c91526625..58a2bd2e542 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java @@ -122,7 +122,7 @@ public class RestServerConfigTest { props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999,"); ConfigException ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props)); - assertTrue(ce.getMessage().contains(" admin.listeners")); + assertTrue(ce.getMessage().contains("admin.listeners")); } @Test diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java index 4d8c25932fe..c4d52fe4b4e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java @@ -48,6 +48,7 @@ public class StandaloneConfigTest { private Map baseWorkerProps() { return new HashMap<>() { { + put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, "/tmp/foo"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index ab261494727..139369c0d56 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -19,6 +19,7 @@ package org.apache.kafka.connect.storage; import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.util.Callback; @@ -84,6 +85,7 @@ public class FileOffsetBackingStoreTest { props.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); props.put(StandaloneConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); props.put(StandaloneConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); config = new StandaloneConfig(props); store.configure(config); store.start(); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java index c1d20a48c1d..cd87c33a509 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/DropHeaders.java @@ -23,7 +23,6 @@ import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashSet; @@ -41,7 +40,8 @@ public class DropHeaders> implements Transformation> implements Transform private static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Names of fields to mask.") .define(REPLACEMENT_CONFIG, ConfigDef.Type.STRING, null, new ConfigDef.NonEmptyString(), ConfigDef.Importance.LOW, "Custom value replacement, that will be applied to all" diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java index 9584ecfc978..7e8f6700bf6 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java @@ -55,9 +55,17 @@ public abstract class ReplaceField> implements Transf } public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(ConfigName.EXCLUDE, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.MEDIUM, + .define(ConfigName.EXCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.MEDIUM, "Fields to exclude. This takes precedence over the fields to include.") - .define(ConfigName.INCLUDE, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.MEDIUM, + .define(ConfigName.INCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.MEDIUM, "Fields to include. If specified, only these fields will be used.") .define(ConfigName.RENAMES, ConfigDef.Type.LIST, List.of(), ConfigDef.LambdaValidator.with( diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java index 24cdec2249a..19c299e6867 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -28,7 +28,6 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; @@ -46,7 +45,7 @@ public class ValueToKey> implements Transformation public static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Field names on the record value to extract as the record key.") .define(REPLACE_NULL_WITH_DEFAULT_CONFIG, ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.MEDIUM, "Whether to replace fields that have a default value and that are null to the default value. When set to true, the default value is used, otherwise null is used."); diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index 83973ede23d..fe1050222b1 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -33,8 +33,8 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)).asScala + val reporters: Seq[String] = (if (props.containsKey(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG)) + Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG)) else MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT).asScala /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 03b2b214a49..671bee32d28 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -222,14 +222,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) case role => throw new ConfigException(s"Unknown process role '$role'" + " (only 'broker' and 'controller' are allowed roles)") } - - val distinctRoles: Set[ProcessRole] = roles.toSet - - if (distinctRoles.size != roles.size) { - throw new ConfigException(s"Duplicate role names found in `${KRaftConfigs.PROCESS_ROLES_CONFIG}`: $roles") - } - - distinctRoles + roles.toSet } def isKRaftCombinedMode: Boolean = { @@ -275,10 +268,10 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) val earlyStartListeners: Set[ListenerName] = { val listenersSet = listeners.map(l => ListenerName.normalised(l.listener)).toSet val controllerListenersSet = controllerListeners.map(l => ListenerName.normalised(l.listener)).toSet - Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { + Option(getList(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { case None => controllerListenersSet - case Some(str) => - str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str => + case Some(list) => + list.asScala.map(_.trim()).filterNot(_.isEmpty).map { str => val listenerName = new ListenerName(str) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " + @@ -442,7 +435,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } def listeners: Seq[Endpoint] = - CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap) + CoreUtils.listenerListToEndPoints(getList(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap) def controllerListeners: Seq[Endpoint] = listeners.filter(l => controllerListenerNames.contains(l.listener)) @@ -457,7 +450,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } def effectiveAdvertisedControllerListeners: Seq[Endpoint] = { - val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) + val advertisedListenersProp = getList(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val controllerAdvertisedListeners = if (advertisedListenersProp != null) { CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) .filter(l => controllerListenerNames.contains(l.listener)) @@ -487,7 +480,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) def effectiveAdvertisedBrokerListeners: Seq[Endpoint] = { // Use advertised listeners if defined, fallback to listeners otherwise - val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) + val advertisedListenersProp = getList(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListeners = if (advertisedListenersProp != null) { CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) } else { diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 31a8aad7c80..66f9bd48657 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -121,16 +121,16 @@ object CoreUtils { def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun) - def listenerListToEndPoints(listeners: String, securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol]): Seq[Endpoint] = { + def listenerListToEndPoints(listeners: java.util.List[String], securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol]): Seq[Endpoint] = { listenerListToEndPoints(listeners, securityProtocolMap, requireDistinctPorts = true) } - private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: String): Unit = { + private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: java.util.List[String]): Unit = { val distinctPorts = endpoints.map(_.port).distinct require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: $listeners") } - def listenerListToEndPoints(listeners: String, securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = { + def listenerListToEndPoints(listeners: java.util.List[String], securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = { def validateOneIsIpv4AndOtherIpv6(first: String, second: String): Boolean = (inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) || (inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second)) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 55911c1e796..dfc999d7c64 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -142,7 +142,7 @@ class KRaftClusterTest { cluster.format() cluster.startup() cluster.brokers().forEach((_, broker) => { - assertEquals("external://localhost:0", broker.config.get(SocketServerConfigs.LISTENERS_CONFIG)) + assertEquals(util.List.of("external://localhost:0"), broker.config.get(SocketServerConfigs.LISTENERS_CONFIG)) assertEquals("external", broker.config.get(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG)) assertEquals("external:PLAINTEXT,CONTROLLER:PLAINTEXT", broker.config.get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)) }) diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 3d5837b92d0..6af0932690f 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -263,8 +263,10 @@ abstract class QuorumTestHarness extends Logging { val listeners = extraControllerSecurityProtocols().map(sc => sc + "://localhost:0").mkString(",") val listenerNames = extraControllerSecurityProtocols().mkString(",") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto,$securityProtocolMaps") - props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0,$listeners") - props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, s"CONTROLLER,$listenerNames") + props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, + if (listeners.isEmpty) "CONTROLLER://localhost:0" else s"CONTROLLER://localhost:0,$listeners") + props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, + if (listeners.isEmpty) "CONTROLLER" else s"CONTROLLER,$listenerNames") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0") props.setProperty(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") val config = new KafkaConfig(props) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 81f2c8e5e08..9ca8b42cd14 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -56,14 +56,14 @@ class KafkaConfigTest { "Invalid value -1 for configuration node.id: Value must be at least 0") properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + assertBadConfigContainingMessage(properties, + "Missing required configuration \"controller.listener.names\" which has no default value.") + + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertBadConfigContainingMessage(properties, "If using process.roles, either controller.quorum.bootstrap.servers must contain the set of bootstrap controllers or controller.quorum.voters must contain a parseable set of controllers.") properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") - assertBadConfigContainingMessage(properties, - "requirement failed: controller.listener.names must contain at least one value when running KRaft with just the broker role") - - properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") KafkaConfig.fromProps(properties) } @@ -82,6 +82,10 @@ class KafkaConfigTest { "Invalid value -1 for configuration node.id: Value must be at least 0") properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + assertBadConfigContainingMessage(properties, + "Missing required configuration \"controller.listener.names\" which has no default value.") + + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertBadConfigContainingMessage(properties, "If using process.roles, either controller.quorum.bootstrap.servers must contain the set of bootstrap controllers or controller.quorum.voters must contain a parseable set of controllers.") @@ -90,15 +94,34 @@ class KafkaConfigTest { "requirement failed: The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") - assertBadConfigContainingMessage(properties, - "No security protocol defined for listener CONTROLLER") - properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT") + KafkaConfig.fromProps(properties) + } + + @Test + def testControllerListenerNamesMismatch(): Unit = { + val properties = new Properties() + properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") + properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "OTHER") + properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") + properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT") + assertBadConfigContainingMessage(properties, "requirement failed: The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") + } - properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") - KafkaConfig.fromProps(properties) + @Test + def testControllerSecurityProtocolMapMissing(): Unit = { + val properties = new Properties() + properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") + properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "OTHER") + properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") + + assertBadConfigContainingMessage(properties, "No security protocol defined for listener CONTROLLER") } @Test diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 1b5c61fc777..e23e16fa40a 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -295,10 +295,10 @@ class LogConfigTest { assertThrows(classOf[ConfigException], () => validateCleanupPolicy()) logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact,delete") assertThrows(classOf[ConfigException], () => validateCleanupPolicy()) - logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,delete,delete") + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete") validateCleanupPolicy() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "") - assertThrows(classOf[ConfigException], () => validateCleanupPolicy()) + validateCleanupPolicy() } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index da54113ae5c..c4cb7a9f7b4 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3187,6 +3187,60 @@ class UnifiedLogTest { assertEquals(segments, log.numberOfSegments, "There should be 3 segments remaining") } + @Test + def shouldDeleteLocalLogSegmentsWhenPolicyIsEmptyWithSizeRetention(): Unit = { + def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) + val recordSize = createRecords.sizeInBytes + val logConfig = LogTestUtils.createLogConfig( + segmentBytes = recordSize * 2, + localRetentionBytes = recordSize / 2, + cleanupPolicy = "", + remoteLogStorageEnable = true + ) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) + + for (_ <- 0 until 10) + log.appendAsLeader(createRecords, 0) + + val segmentsBefore = log.numberOfSegments + log.updateHighWatermark(log.logEndOffset) + log.updateHighestOffsetInRemoteStorage(log.logEndOffset - 1) + val deleteOldSegments = log.deleteOldSegments() + + assertTrue(log.numberOfSegments < segmentsBefore, "Some segments should be deleted due to size retention") + assertTrue(deleteOldSegments > 0, "At least one segment should be deleted") + } + + @Test + def shouldDeleteLocalLogSegmentsWhenPolicyIsEmptyWithMsRetention(): Unit = { + val oldTimestamp = mockTime.milliseconds - 20000 + def oldRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = oldTimestamp) + val recordSize = oldRecords.sizeInBytes + val logConfig = LogTestUtils.createLogConfig( + segmentBytes = recordSize * 2, + localRetentionMs = 5000, + cleanupPolicy = "", + remoteLogStorageEnable = true + ) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) + + for (_ <- 0 until 10) + log.appendAsLeader(oldRecords, 0) + + def newRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = mockTime.milliseconds) + for (_ <- 0 until 5) + log.appendAsLeader(newRecords, 0) + + val segmentsBefore = log.numberOfSegments + + log.updateHighWatermark(log.logEndOffset) + log.updateHighestOffsetInRemoteStorage(log.logEndOffset - 1) + val deleteOldSegments = log.deleteOldSegments() + + assertTrue(log.numberOfSegments < segmentsBefore, "Some segments should be deleted due to time retention") + assertTrue(deleteOldSegments > 0, "At least one segment should be deleted") + } + @Test def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index fb93f8fa84b..dc24a369515 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -223,7 +223,7 @@ class KafkaConfigTest { // but not duplicate names props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "HOST://localhost:9091,HOST://localhost:9091") - assertBadConfigContainingMessage(props, "Each listener must have a different name") + assertBadConfigContainingMessage(props, "Configuration 'advertised.listeners' values must not be duplicated.") } @Test @@ -248,8 +248,8 @@ class KafkaConfigTest { assertTrue(caught.getMessage.contains("If you have two listeners on the same port then one needs to be IPv4 and the other IPv6")) props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,PLAINTEXT://127.0.0.1:9092") - caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) - assertTrue(caught.getMessage.contains("Each listener must have a different name")) + val exception = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) + assertTrue(exception.getMessage.contains("values must not be duplicated.")) props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9092,SASL_SSL://127.0.0.1:9092") caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) @@ -301,7 +301,8 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") - assertBadConfigContainingMessage(props, "The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") + assertBadConfigContainingMessage(props, + "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") KafkaConfig.fromProps(props) @@ -321,7 +322,8 @@ class KafkaConfigTest { props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) - assertBadConfigContainingMessage(props, "controller.listener.names must contain at least one value when running KRaft with just the broker role") + assertBadConfigContainingMessage(props, + "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") KafkaConfig.fromProps(props) @@ -607,7 +609,7 @@ class KafkaConfigTest { assertEquals(expected.securityProtocol(), actual.securityProtocol(), "Security protocol mismatch") } - private def listenerListToEndPoints(listenerList: String, + private def listenerListToEndPoints(listenerList: java.util.List[String], securityProtocolMap: util.Map[ListenerName, SecurityProtocol] = SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO) = CoreUtils.listenerListToEndPoints(listenerList, securityProtocolMap) @@ -621,9 +623,9 @@ class KafkaConfigTest { // configuration with no listeners val conf = KafkaConfig.fromProps(props) - assertEquals(listenerListToEndPoints("PLAINTEXT://:9092"), conf.listeners) + assertEquals(listenerListToEndPoints(util.List.of("PLAINTEXT://:9092")), conf.listeners) assertNull(conf.listeners.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get.host) - assertEquals(conf.effectiveAdvertisedBrokerListeners, listenerListToEndPoints("PLAINTEXT://:9092")) + assertEquals(conf.effectiveAdvertisedBrokerListeners, listenerListToEndPoints(util.List.of("PLAINTEXT://:9092"))) } private def isValidKafkaConfig(props: Properties): Boolean = { @@ -827,8 +829,8 @@ class KafkaConfigTest { case SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.NUM_PARTITIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string - case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string + case ServerLogConfigs.LOG_DIRS_CONFIG => assertPropertyInvalid(baseProperties, name, "") + case ServerLogConfigs.LOG_DIR_CONFIG => assertPropertyInvalid(baseProperties, name, "") case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") @@ -1600,6 +1602,7 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") + props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertEquals("You must set `node.id` to the same value as `broker.id`.", assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) } diff --git a/docs/configuration.html b/docs/configuration.html index 425d306a4dc..0c114f79ef4 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -26,6 +26,7 @@
  • log.dirs
  • process.roles
  • controller.quorum.bootstrap.servers +
  • controller.listener.names Topic configurations and defaults are discussed in more detail below. diff --git a/docs/upgrade.html b/docs/upgrade.html index 38ed0963af1..c7af07e0411 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -113,6 +113,35 @@
  • The num.replica.fetchers config has a new lower bound of 1.
  • +
  • + Improvements have been made to the validation rules and default values of LIST-type configurations + (KIP-1161). +
      +
    • + LIST-type configurations now enforce stricter validation: +
        +
      • Null values are no longer accepted for most LIST-type configurations, except those that explicitly + allow a null default value or where a null value has a well-defined semantic meaning.
      • +
      • Duplicate entries within the same list are no longer permitted.
      • +
      • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined + semantic meaning.
      • +
      +
    • +
    • + Several configurations have been reclassified from STRING-type to LIST-type to better reflect their + intended use as comma-separated values. +
    • +
    • + Default values for certain configurations have been adjusted to ensure better consistency with related + settings. +
    • +
    • + The cleanup.policy is empty and remote.storage.enable is set to true, the + local log segments will be cleaned based on the values of log.local.retention.bytes and + log.local.retention.ms. +
    • +
    +
  • Upgrading to 4.1.0

    diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java index f3adcae3fc5..11dd3e30e3f 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -298,7 +298,7 @@ public class GroupCoordinatorConfig { public static final ConfigDef CONFIG_DEF = new ConfigDef() // Group coordinator configs .define(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, - ConfigDef.ValidList.in(Group.GroupType.documentValidValues()), MEDIUM, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) + ConfigDef.ValidList.in(false, Group.GroupType.documentValidValues()), MEDIUM, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), HIGH, GROUP_COORDINATOR_NUM_THREADS_DOC) .define(GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) .define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, OFFSET_COMMIT_TIMEOUT_MS_DOC) @@ -327,7 +327,7 @@ public class GroupCoordinatorConfig { .define(CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DOC) .define(CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC) .define(CONSUMER_GROUP_MAX_SIZE_CONFIG, INT, CONSUMER_GROUP_MAX_SIZE_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MAX_SIZE_DOC) - .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) + .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) .define(CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(ConsumerGroupMigrationPolicy.class)), MEDIUM, CONSUMER_GROUP_MIGRATION_POLICY_DOC) // Interval config used for testing purposes. .defineInternal(CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DEFAULT, atLeast(10 * 1000), MEDIUM, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DOC) @@ -340,7 +340,7 @@ public class GroupCoordinatorConfig { .define(SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DOC) .define(SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC) .define(SHARE_GROUP_MAX_SIZE_CONFIG, INT, SHARE_GROUP_MAX_SIZE_DEFAULT, between(1, 1000), MEDIUM, SHARE_GROUP_MAX_SIZE_DOC) - .define(SHARE_GROUP_ASSIGNORS_CONFIG, LIST, SHARE_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, SHARE_GROUP_ASSIGNORS_DOC) + .define(SHARE_GROUP_ASSIGNORS_CONFIG, LIST, SHARE_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, SHARE_GROUP_ASSIGNORS_DOC) .defineInternal(SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_DEFAULT, atLeast(1), LOW, SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_DOC) // Streams group configs diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java index 4910f00016b..e40c5c8e6d3 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerConfigs.java @@ -33,6 +33,7 @@ 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; import static org.apache.kafka.common.config.ConfigDef.Type.LONG; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; @@ -133,10 +134,10 @@ public class ServerConfigs { .define(REQUEST_TIMEOUT_MS_CONFIG, INT, REQUEST_TIMEOUT_MS_DEFAULT, HIGH, REQUEST_TIMEOUT_MS_DOC) .define(SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, LONG, DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) .define(SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, LONG, SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) /************* Authorizer Configuration ***********/ .define(AUTHORIZER_CLASS_NAME_CONFIG, STRING, AUTHORIZER_CLASS_NAME_DEFAULT, new ConfigDef.NonNullValidator(), LOW, AUTHORIZER_CLASS_NAME_DOC) - .define(EARLY_START_LISTENERS_CONFIG, STRING, null, HIGH, EARLY_START_LISTENERS_DOC) + .define(EARLY_START_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyNonDuplicateValues(true, true), HIGH, EARLY_START_LISTENERS_DOC) /************ Rack Configuration ******************/ .define(BROKER_RACK_CONFIG, STRING, null, MEDIUM, BROKER_RACK_DOC) /** ********* Controlled shutdown configuration ***********/ diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012..5438a0a59cb 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -37,7 +37,7 @@ public class ServerLogConfigs { public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs"; public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir"; public static final String LOG_DIR_DEFAULT = "/tmp/kafka-logs"; - public static final String LOG_DIR_DOC = "The directory in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)"; + public static final String LOG_DIR_DOC = "A comma-separated list of the directories where the log data is stored. (supplemental to " + LOG_DIRS_CONFIG + " property)"; public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used."; public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); @@ -71,7 +71,7 @@ public class ServerLogConfigs { public static final String LOG_CLEANUP_POLICY_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG); public static final String LOG_CLEANUP_POLICY_DEFAULT = TopicConfig.CLEANUP_POLICY_DELETE; - public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies."; + public static final String LOG_CLEANUP_POLICY_DOC = TopicConfig.CLEANUP_POLICY_DOC; public static final String LOG_INDEX_SIZE_MAX_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); public static final int LOG_INDEX_SIZE_MAX_BYTES_DEFAULT = 10 * 1024 * 1024; diff --git a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java index e9c3dfff926..5aa0ddcf944 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.server.util.Csv; import java.util.ArrayList; import java.util.Arrays; @@ -39,6 +38,7 @@ 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.INT; +import static org.apache.kafka.common.config.ConfigDef.Type.LIST; import static org.apache.kafka.common.config.ConfigDef.Type.LONG; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; @@ -154,9 +154,9 @@ public class SocketServerConfigs { public static final int NUM_NETWORK_THREADS_DEFAULT = 3; public static final String NUM_NETWORK_THREADS_DOC = "The number of threads that the server uses for receiving requests from the network and sending responses to the network. Noted: each listener (except for controller listener) creates its own thread pool."; - public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(LISTENERS_CONFIG, STRING, LISTENERS_DEFAULT, HIGH, LISTENERS_DOC) - .define(ADVERTISED_LISTENERS_CONFIG, STRING, null, HIGH, ADVERTISED_LISTENERS_DOC) + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(LISTENERS_CONFIG, LIST, LISTENERS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, LISTENERS_DOC) + .define(ADVERTISED_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyNonDuplicateValues(false, true), HIGH, ADVERTISED_LISTENERS_DOC) .define(LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, STRING, LISTENER_SECURITY_PROTOCOL_MAP_DEFAULT, LOW, LISTENER_SECURITY_PROTOCOL_MAP_DOC) .define(SOCKET_SEND_BUFFER_BYTES_CONFIG, INT, SOCKET_SEND_BUFFER_BYTES_DEFAULT, HIGH, SOCKET_SEND_BUFFER_BYTES_DOC) .define(SOCKET_RECEIVE_BUFFER_BYTES_CONFIG, INT, SOCKET_RECEIVE_BUFFER_BYTES_DEFAULT, HIGH, SOCKET_RECEIVE_BUFFER_BYTES_DOC) @@ -186,7 +186,7 @@ public class SocketServerConfigs { } public static List listenerListToEndPoints( - String input, + List input, Map nameToSecurityProto ) { return listenerListToEndPoints(input, n -> { @@ -199,11 +199,11 @@ public class SocketServerConfigs { } public static List listenerListToEndPoints( - String input, + List input, Function nameToSecurityProto ) { List results = new ArrayList<>(); - for (String entry : Csv.parseCsvList(input.trim())) { + for (String entry : input) { Matcher matcher = URI_PARSE_REGEXP.matcher(entry); if (!matcher.matches()) { throw new KafkaException("Unable to parse " + entry + " to a broker endpoint"); diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 16d61722727..7be9fcf23f5 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -79,7 +79,7 @@ public abstract class AbstractKafkaConfig extends AbstractConfig { } public List logDirs() { - return Csv.parseCsvList(Optional.ofNullable(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).orElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))); + return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)).orElse(getList(ServerLogConfigs.LOG_DIR_CONFIG)); } public int numIoThreads() { @@ -107,7 +107,7 @@ public abstract class AbstractKafkaConfig extends AbstractConfig { } public List controllerListenerNames() { - return Csv.parseCsvList(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)); + return getList(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG); } public ListenerName interBrokerListenerName() { @@ -137,7 +137,7 @@ public abstract class AbstractKafkaConfig extends AbstractConfig { // 2. No SSL or SASL protocols are used in regular listeners (Note: controller listeners // are not included in 'listeners' config when process.roles=broker) if (controllerListenerNames().stream().anyMatch(AbstractKafkaConfig::isSslOrSasl) || - Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).stream() + getList(SocketServerConfigs.LISTENERS_CONFIG).stream() .anyMatch(listenerName -> isSslOrSasl(parseListenerName(listenerName)))) { return mapValue; } else { diff --git a/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java b/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java index 9e9580e6d75..365f0937cd8 100644 --- a/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java @@ -70,12 +70,12 @@ public class KRaftConfigs { public static final String CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DOC = "We will log an error message about controller events that take longer than this threshold."; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(PROCESS_ROLES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.in("broker", "controller"), HIGH, PROCESS_ROLES_DOC) + .define(PROCESS_ROLES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.in(false, "broker", "controller"), HIGH, PROCESS_ROLES_DOC) .define(NODE_ID_CONFIG, INT, ConfigDef.NO_DEFAULT_VALUE, atLeast(0), HIGH, NODE_ID_DOC) .define(INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, INT, INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT, null, MEDIUM, INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC) .define(BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, INT, BROKER_HEARTBEAT_INTERVAL_MS_DEFAULT, null, MEDIUM, BROKER_HEARTBEAT_INTERVAL_MS_DOC) .define(BROKER_SESSION_TIMEOUT_MS_CONFIG, INT, BROKER_SESSION_TIMEOUT_MS_DEFAULT, null, MEDIUM, BROKER_SESSION_TIMEOUT_MS_DOC) - .define(CONTROLLER_LISTENER_NAMES_CONFIG, STRING, null, null, HIGH, CONTROLLER_LISTENER_NAMES_DOC) + .define(CONTROLLER_LISTENER_NAMES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, CONTROLLER_LISTENER_NAMES_DOC) .define(SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SASL_MECHANISM_CONTROLLER_PROTOCOL_DOC) .defineInternal(CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS, LONG, CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS_DEFAULT, atLeast(100), MEDIUM, CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS_DOC) .defineInternal(CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS, LONG, CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DEFAULT, atLeast(0), MEDIUM, CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DOC) diff --git a/server/src/main/java/org/apache/kafka/server/metrics/ClientMetricsConfigs.java b/server/src/main/java/org/apache/kafka/server/metrics/ClientMetricsConfigs.java index 2560854eb55..b7a66df7b16 100644 --- a/server/src/main/java/org/apache/kafka/server/metrics/ClientMetricsConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/metrics/ClientMetricsConfigs.java @@ -102,9 +102,19 @@ public class ClientMetricsConfigs extends AbstractConfig { ); private static final ConfigDef CONFIG = new ConfigDef() - .define(METRICS_CONFIG, Type.LIST, METRICS_DEFAULT, Importance.MEDIUM, "Telemetry metric name prefix list") + .define(METRICS_CONFIG, + Type.LIST, + METRICS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.MEDIUM, + "Telemetry metric name prefix list") .define(INTERVAL_MS_CONFIG, Type.INT, INTERVAL_MS_DEFAULT, Importance.MEDIUM, "Metrics push interval in milliseconds") - .define(MATCH_CONFIG, Type.LIST, MATCH_DEFAULT, Importance.MEDIUM, "Client match criteria"); + .define(MATCH_CONFIG, + Type.LIST, + MATCH_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.MEDIUM, + "Client match criteria"); public ClientMetricsConfigs(Properties props) { super(CONFIG, props, false); diff --git a/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java b/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java index 91aca815287..a974c36bed5 100644 --- a/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.Sensor; +import java.util.List; + import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Type.INT; @@ -48,7 +50,7 @@ public class MetricConfigs { /** ********* Kafka Yammer Metrics Reporters Configuration ***********/ public static final String KAFKA_METRICS_REPORTER_CLASSES_CONFIG = "kafka.metrics.reporters"; - public static final String KAFKA_METRIC_REPORTER_CLASSES_DEFAULT = ""; + public static final List KAFKA_METRIC_REPORTER_CLASSES_DEFAULT = List.of(); public static final String KAFKA_METRICS_REPORTER_CLASSES_DOC = "A list of classes to use as Yammer metrics custom reporters." + " The reporters should implement kafka.metrics.KafkaMetricsReporter trait. If a client wants" + " to expose JMX operations on a custom reporter, the custom reporter needs to additionally implement an MBean" + @@ -69,11 +71,11 @@ public class MetricConfigs { // Kafka Metrics Configuration .define(METRIC_NUM_SAMPLES_CONFIG, INT, METRIC_NUM_SAMPLES_DEFAULT, atLeast(1), LOW, METRIC_NUM_SAMPLES_DOC) .define(METRIC_SAMPLE_WINDOW_MS_CONFIG, LONG, METRIC_SAMPLE_WINDOW_MS_DEFAULT, atLeast(1), LOW, METRIC_SAMPLE_WINDOW_MS_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, LIST, METRIC_REPORTER_CLASSES_DEFAULT, LOW, METRIC_REPORTER_CLASSES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, LIST, METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRIC_RECORDING_LEVEL_CONFIG, STRING, METRIC_RECORDING_LEVEL_DEFAULT, LOW, METRIC_RECORDING_LEVEL_DOC) // Kafka Yammer Metrics Reporter Configuration - .define(KAFKA_METRICS_REPORTER_CLASSES_CONFIG, LIST, KAFKA_METRIC_REPORTER_CLASSES_DEFAULT, LOW, KAFKA_METRICS_REPORTER_CLASSES_DOC) + .define(KAFKA_METRICS_REPORTER_CLASSES_CONFIG, LIST, KAFKA_METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, KAFKA_METRICS_REPORTER_CLASSES_DOC) .define(KAFKA_METRICS_POLLING_INTERVAL_SECONDS_CONFIG, INT, KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DEFAULT, atLeast(1), LOW, KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DOC) // Kafka Client Telemetry Metrics Configuration diff --git a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java index be4184a9202..6b0ca02e7cc 100644 --- a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java +++ b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.network; import org.apache.kafka.common.Endpoint; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -26,6 +27,7 @@ import java.util.List; import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class SocketServerConfigsTest { @Test @@ -42,21 +44,21 @@ public class SocketServerConfigsTest { @Test public void testListenerListToEndPointsWithEmptyString() { assertEquals(List.of(), - SocketServerConfigs.listenerListToEndPoints("", + SocketServerConfigs.listenerListToEndPoints(List.of(), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithBlankString() { - assertEquals(List.of(), - SocketServerConfigs.listenerListToEndPoints(" ", - SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); + KafkaException exception = assertThrows(KafkaException.class, () -> + SocketServerConfigs.listenerListToEndPoints(List.of(" "), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); + assertEquals("Unable to parse to a broker endpoint", exception.getMessage()); } @Test public void testListenerListToEndPointsWithOneEndpoint() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 8080)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://example.com:8080"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @@ -66,35 +68,35 @@ public class SocketServerConfigsTest { assertEquals(List.of( new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 8080), new Endpoint("SSL", SecurityProtocol.SSL, "local_host", 8081)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080,SSL://local_host:8081", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://example.com:8080", "SSL://local_host:8081"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithWildcard() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, null, 8080)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://:8080", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://:8080"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithIpV6() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "::1", 9092)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://[::1]:9092", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://[::1]:9092"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testAnotherListenerListToEndPointsWithIpV6() { assertEquals(List.of(new Endpoint("SASL_SSL", SecurityProtocol.SASL_SSL, "fe80::b1da:69ca:57f7:63d8%3", 9092)), - SocketServerConfigs.listenerListToEndPoints("SASL_SSL://[fe80::b1da:69ca:57f7:63d8%3]:9092", + SocketServerConfigs.listenerListToEndPoints(List.of("SASL_SSL://[fe80::b1da:69ca:57f7:63d8%3]:9092"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testAnotherListenerListToEndPointsWithNonDefaultProtoMap() { assertEquals(List.of(new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "example.com", 9093)), - SocketServerConfigs.listenerListToEndPoints("CONTROLLER://example.com:9093", + SocketServerConfigs.listenerListToEndPoints(List.of("CONTROLLER://example.com:9093"), Map.of(new ListenerName("CONTROLLER"), SecurityProtocol.PLAINTEXT))); } } diff --git a/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java b/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java index 985d1834264..90a8132e8ea 100644 --- a/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java @@ -85,7 +85,10 @@ public class AddPartitionsToTxnManagerTest { private final MockTime time = new MockTime(); private final AbstractKafkaConfig config = new AbstractKafkaConfig( AbstractKafkaConfig.CONFIG_DEF, - Map.of(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker", KRaftConfigs.NODE_ID_CONFIG, "1"), + Map.of( + KRaftConfigs.PROCESS_ROLES_CONFIG, "broker", + KRaftConfigs.NODE_ID_CONFIG, "1", + KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER"), Map.of(), false) { }; private final AddPartitionsToTxnManager addPartitionsToTxnManager = diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java index 363fb06c2de..a687f3c529e 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/LogConfig.java @@ -144,8 +144,8 @@ public class LogConfig extends AbstractConfig { public static final ConfigDef SERVER_CONFIG_DEF = new ConfigDef() .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC) - .define(ServerLogConfigs.LOG_DIR_CONFIG, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC) - .define(ServerLogConfigs.LOG_DIRS_CONFIG, STRING, null, HIGH, ServerLogConfigs.LOG_DIRS_DOC) + .define(ServerLogConfigs.LOG_DIR_CONFIG, LIST, ServerLogConfigs.LOG_DIR_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, ServerLogConfigs.LOG_DIR_DOC) + .define(ServerLogConfigs.LOG_DIRS_CONFIG, LIST, null, ConfigDef.ValidList.anyNonDuplicateValues(false, true), HIGH, ServerLogConfigs.LOG_DIRS_DOC) .define(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, INT, DEFAULT_SEGMENT_BYTES, atLeast(1024 * 1024), HIGH, ServerLogConfigs.LOG_SEGMENT_BYTES_DOC) .define(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG, LONG, null, HIGH, ServerLogConfigs.LOG_ROLL_TIME_MILLIS_DOC) @@ -561,9 +561,8 @@ public class LogConfig extends AbstractConfig { @SuppressWarnings("unchecked") private static void validateRemoteStorageRequiresDeleteCleanupPolicy(Map props) { List cleanupPolicy = (List) props.get(TopicConfig.CLEANUP_POLICY_CONFIG); - Set policySet = cleanupPolicy.stream().map(policy -> policy.toLowerCase(Locale.getDefault())).collect(Collectors.toSet()); - if (!Set.of(TopicConfig.CLEANUP_POLICY_DELETE).equals(policySet)) { - throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete"); + if (!cleanupPolicy.isEmpty() && (cleanupPolicy.size() != 1 || !TopicConfig.CLEANUP_POLICY_DELETE.equals(cleanupPolicy.get(0)))) { + throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete or cleanup.policy being an empty list."); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index 769f59d56dc..61b4b9d0edb 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1906,16 +1906,29 @@ public class UnifiedLog implements AutoCloseable { /** * If topic deletion is enabled, delete any local log segments that have either expired due to time based - * retention or because the log size is > retentionSize. Whether or not deletion is enabled, delete any local - * log segments that are before the log start offset + * retention or because the log size is > retentionSize. Empty cleanup.policy is the same as delete with + * infinite retention, so we only need to delete local segments if remote storage is enabled. Whether or + * not deletion is enabled, delete any local log segments that are before the log start offset */ public int deleteOldSegments() throws IOException { if (config().delete) { return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments() + deleteRetentionMsBreachedSegments(); - } else { + } else if (config().compact) { return deleteLogStartOffsetBreachedSegments(); + } else { + // If cleanup.policy is empty and remote storage is enabled, the local log segments will + // be cleaned based on the values of log.local.retention.bytes and log.local.retention.ms + if (remoteLogEnabledAndRemoteCopyEnabled()) { + return deleteLogStartOffsetBreachedSegments() + + deleteRetentionSizeBreachedSegments() + + deleteRetentionMsBreachedSegments(); + } else { + // If cleanup.policy is empty and remote storage is disabled, we should not delete any local + // log segments + return deleteLogStartOffsetBreachedSegments(); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 48822a996d5..1bd533fbd14 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -78,6 +78,7 @@ import java.util.stream.Stream; import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED; import static org.apache.kafka.common.config.ConfigDef.ListSize.atMostOfSize; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import static org.apache.kafka.common.config.ConfigDef.parseType; @@ -901,6 +902,8 @@ public class StreamsConfig extends AbstractConfig { APPLICATION_ID_DOC) .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value Type.LIST, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, @@ -1099,7 +1102,8 @@ public class StreamsConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), - Importance.LOW, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.LOW, CONFIG_PROVIDERS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, Type.BOOLEAN, @@ -1175,6 +1179,7 @@ public class StreamsConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index f850093d389..3b795310b36 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -139,7 +139,7 @@ public class StateDirectoryTest { assertPermissions(stateDir); assertPermissions(appDir); } - + private void assertPermissions(final File file) { final Path path = file.toPath(); if (path.getFileSystem().supportedFileAttributeViews().contains("posix")) { @@ -630,7 +630,7 @@ public class StateDirectoryTest { new StateDirectory( new StreamsConfig( mkMap( - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""), + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"), mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "") ) ), @@ -1004,4 +1004,4 @@ public class StateDirectoryTest { } } } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 56c19c98529..0a65d47d207 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -98,7 +98,7 @@ public class TimeOrderedKeyValueBufferTest makeContext() { final Properties properties = new Properties(); properties.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID); - properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock:localhost:9092"); final TaskId taskId = new TaskId(0, 0); diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index fc7d27a3bb7..0ffea9b4916 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -194,7 +194,7 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S new Properties() { { put(StreamsConfig.APPLICATION_ID_CONFIG, ""); - put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"); } }, new TaskId(0, 0), diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java index 52a2308dafe..3ffa85a4503 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java @@ -214,7 +214,7 @@ public class MockProcessorContext implements ProcessorContex this( mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy-bootstrap-host:0") )), new TaskId(0, 0), null diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java index 929b5d6533d..f2b7f8ee68c 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java @@ -242,7 +242,7 @@ public class MockProcessorContextAPITest { final Properties config = mkProperties( mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "testMetadata"), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092") ) ); @@ -336,7 +336,7 @@ public class MockProcessorContextAPITest { public void fullConstructorShouldSetAllExpectedAttributes() { final Properties config = new Properties(); config.put(StreamsConfig.APPLICATION_ID_CONFIG, "testFullConstructor"); - config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"); config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class.getName()); config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.LongSerde.class.getName()); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java index 521a39d692b..29e19b4ce0a 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java @@ -166,7 +166,7 @@ public class MockProcessorContextStateStoreTest { final MockProcessorContext context = new MockProcessorContext<>( mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092") )), new TaskId(0, 0), stateDir diff --git a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java index 13c30721a57..9090fe46e47 100644 --- a/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/BrokerApiVersionsCommand.java @@ -144,7 +144,7 @@ public class BrokerApiVersionsCommand { private static final AtomicInteger ADMIN_CLIENT_ID_SEQUENCE = new AtomicInteger(1); private static final ConfigDef ADMIN_CONFIG_DEF = new ConfigDef() - .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, ConfigDef.Type.LIST, ConfigDef.Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG, ConfigDef.Type.STRING, ClientDnsLookup.USE_ALL_DNS_IPS.toString(), ConfigDef.ValidString.in(ClientDnsLookup.USE_ALL_DNS_IPS.toString(), ClientDnsLookup.RESOLVE_CANONICAL_BOOTSTRAP_SERVERS_ONLY.toString()), ConfigDef.Importance.MEDIUM, CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, ConfigDef.Type.STRING, CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(SecurityProtocol.class)), ConfigDef.Importance.MEDIUM, CommonClientConfigs.SECURITY_PROTOCOL_DOC) .define(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG, ConfigDef.Type.INT, DEFAULT_REQUEST_TIMEOUT_MS, ConfigDef.Importance.MEDIUM, CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC) diff --git a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java index 4abe443b82b..e42dcbb0e09 100644 --- a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java @@ -33,6 +33,7 @@ import org.apache.kafka.raft.MetadataLogConfig; import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.CommandLineUtils; +import org.apache.kafka.server.util.Csv; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.impl.Arguments; @@ -395,10 +396,10 @@ public class MetadataQuorumCommand { ) throws Exception { Map listeners = new HashMap<>(); SocketServerConfigs.listenerListToEndPoints( - props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG, "").toString(), + Csv.parseCsvList(props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG, "").toString()), __ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e)); SocketServerConfigs.listenerListToEndPoints( - props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "").toString(), + Csv.parseCsvList(props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "").toString()), __ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e)); if (!props.containsKey(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)) { throw new TerseException(KRaftConfigs.CONTROLLER_LISTENER_NAMES_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 fe9028b8c41..c6f1073edb5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -885,7 +885,7 @@ public class TopicCommandTest { public void testTopicDeletion(ClusterInstance clusterInstance) throws Exception { try (Admin adminClient = clusterInstance.admin(); TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { - String testTopicName = TestUtils.randomString(10); + String testTopicName = "testing"; adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions);