From 168a1cd6496d44563fe2a388cc3ed88013debc8a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 7 Nov 2025 08:39:01 +0800 Subject: [PATCH 01/16] add new test case for testListValidatorAnyNonDuplicateValues --- .../kafka/common/config/ConfigDefTest.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) 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 c6c2390b07c47..1cbb88ba62ba7 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 @@ -760,15 +760,15 @@ public void testListSizeValidatorToString() { @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"))); + ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyListAndNull = ConfigDef.ValidList.anyNonDuplicateValues(true, true); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "b", "c"))); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of())); + assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", null)); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.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(""))); + ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.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)); @@ -788,6 +788,17 @@ public void testListValidatorAnyNonDuplicateValues() { 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()); + + ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyNonDuplicateValues(false, false); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "b", "c"))); + ConfigException exception9 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception9.getMessage()); + ConfigException exception10 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception10.getMessage()); + ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception11.getMessage()); + ConfigException exception12 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception12.getMessage()); } @Test From 84f001cc19edb721b559878835c19f367ced8f39 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 8 Nov 2025 10:15:42 +0800 Subject: [PATCH 02/16] addressed by comments --- docs/upgrade.html | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index c0e5d6adf8ce8..09f6e5f6cdfab 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -133,7 +133,8 @@
Notable changes in 4
  • 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.
  • +
  • Most LIST-type configurations no longer accept duplicate entries, except in cases where duplicates + are explicitly supported.
  • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined semantic meaning.
From 8d9a7f072bb74e825786fe2166ddc9f4f348b890 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 10 Nov 2025 19:52:16 +0800 Subject: [PATCH 03/16] warnings the duplicate value --- .../apache/kafka/common/config/ConfigDef.java | 3 ++- .../kafka/common/config/ConfigDefTest.java | 16 ++++++++-------- .../main/scala/kafka/server/KafkaConfig.scala | 7 ++++++- .../scala/unit/kafka/log/LogConfigTest.scala | 2 +- .../unit/kafka/server/KafkaConfigTest.scala | 10 +++++----- .../kafka/storage/internals/log/LogConfig.java | 3 ++- 6 files changed, 24 insertions(+), 17 deletions(-) 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 ee2f8c2cfd951..5ae8204ba42f8 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 @@ -1047,7 +1047,8 @@ public void ensureValid(final String name, final Object value) { } if (Set.copyOf(values).size() != values.size()) { - throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); + System.out.println("Configuration '" + name + "' has duplicate values: " + values + + "this will be disallowed in Kafka5.0."); } validateIndividualValues(name, values); 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 1cbb88ba62ba7..6a6a5606ff417 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 @@ -764,8 +764,8 @@ public void testListValidatorAnyNonDuplicateValues() { assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "b", "c"))); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of())); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", null)); - ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "a"))); - assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception1.getMessage()); ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); @@ -774,8 +774,8 @@ public void testListValidatorAnyNonDuplicateValues() { 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 exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception4.getMessage()); ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception5.getMessage()); @@ -784,8 +784,8 @@ public void testListValidatorAnyNonDuplicateValues() { 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 exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception7.getMessage()); ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception8.getMessage()); @@ -795,8 +795,8 @@ public void testListValidatorAnyNonDuplicateValues() { assertEquals("Configuration 'test.config' values must not be null.", exception9.getMessage()); ConfigException exception10 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception10.getMessage()); - ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "a"))); - assertEquals("Configuration 'test.config' values must not be duplicated.", exception11.getMessage()); + ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception11.getMessage()); ConfigException exception12 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception12.getMessage()); } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a7317a20e896d..f9b8e05ef9ef9 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -221,7 +221,12 @@ 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)") } - roles.toSet + val distinctRoles: Set[ProcessRole] = roles.toSet + + if (distinctRoles.size != roles.size) { + throw new ConfigException(s"Configuration '${KRaftConfigs.PROCESS_ROLES_CONFIG}' values must not be duplicated.") + } + distinctRoles } def isKRaftCombinedMode: Boolean = { diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index e942e7e33805c..98a49070913b1 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -294,7 +294,7 @@ 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") + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete,delete,delete") validateCleanupPolicy() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "") validateCleanupPolicy() diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index ab7f45ae6895e..157c46ae3e877 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, "Configuration 'advertised.listeners' values must not be duplicated.") + assertBadConfigContainingMessage(props, "Each listener must have a different name") } @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") - val exception = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) - assertTrue(exception.getMessage.contains("values must not be duplicated.")) + caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) + assertTrue(caught.getMessage.contains("Each listener must have a different name")) 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,7 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") - assertBadConfigContainingMessage(props, + assertBadConfigContainingMessage(props, "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") @@ -322,7 +322,7 @@ class KafkaConfigTest { props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) - assertBadConfigContainingMessage(props, + assertBadConfigContainingMessage(props, "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") 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 a687f3c529e32..5d0bec2745304 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 @@ -561,7 +561,8 @@ public static void validateRemoteStorageOnlyIfSystemEnabled(Map props, boo @SuppressWarnings("unchecked") private static void validateRemoteStorageRequiresDeleteCleanupPolicy(Map props) { List cleanupPolicy = (List) props.get(TopicConfig.CLEANUP_POLICY_CONFIG); - if (!cleanupPolicy.isEmpty() && (cleanupPolicy.size() != 1 || !TopicConfig.CLEANUP_POLICY_DELETE.equals(cleanupPolicy.get(0)))) { + Set policySet = cleanupPolicy.stream().map(policy -> policy.toLowerCase(Locale.getDefault())).collect(Collectors.toSet()); + if (!policySet.isEmpty() && (policySet.size() != 1 || !Set.of(TopicConfig.CLEANUP_POLICY_DELETE).equals(policySet))) { throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete or cleanup.policy being an empty list."); } } From 902d071b4ffde04f29527e404bd06c82d13f23a2 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 10 Nov 2025 22:31:55 +0800 Subject: [PATCH 04/16] fix fail test --- .../kafka/common/config/ConfigDefTest.java | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) 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 6a6a5606ff417..c75e08779cc53 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 @@ -810,18 +810,14 @@ public void testListValidatorIn() { 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()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: [a, b, c]", exception3.getMessage()); + ConfigException exception4 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception4.getMessage()); + ConfigException exception5 = 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", exception5.getMessage()); } } From 820fd4a7af7ba03efe4f7da080ec69171fa2918e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:17:59 +0800 Subject: [PATCH 05/16] addressed by comments --- .../clients/admin/AdminClientConfig.java | 12 +++++------ .../clients/consumer/ConsumerConfig.java | 10 +++++----- .../clients/producer/ProducerConfig.java | 8 ++++---- .../kafka/common/config/AbstractConfig.java | 16 +++++++++------ .../apache/kafka/common/config/ConfigDef.java | 18 +++++++++-------- .../kafka/common/config/SaslConfigs.java | 2 +- .../kafka/common/config/SslConfigs.java | 8 ++++---- .../internals/BrokerSecurityConfigs.java | 2 +- .../kafka/common/config/ConfigDefTest.java | 8 ++++---- .../connect/mirror/MirrorClientConfig.java | 4 ++-- .../connect/mirror/DefaultGroupFilter.java | 4 ++-- .../mirror/MirrorCheckpointConfig.java | 4 ++-- .../mirror/MirrorCheckpointTaskConfig.java | 2 +- .../connect/mirror/MirrorConnectorConfig.java | 6 +++--- .../connect/mirror/MirrorMakerConfig.java | 20 +++++++++---------- .../connect/mirror/MirrorSourceConfig.java | 8 ++++---- .../mirror/MirrorSourceTaskConfig.java | 4 ++-- .../connect/runtime/SinkConnectorConfig.java | 2 +- .../kafka/connect/runtime/WorkerConfig.java | 8 ++++---- .../runtime/rest/RestServerConfig.java | 4 ++-- .../kafka/connect/transforms/DropHeaders.java | 4 ++-- .../kafka/connect/transforms/MaskField.java | 4 ++-- .../connect/transforms/ReplaceField.java | 16 +++++++-------- .../kafka/connect/transforms/ValueToKey.java | 2 +- .../connect/transforms/util/SimpleConfig.java | 4 ++++ .../main/scala/kafka/server/KafkaConfig.scala | 7 +------ .../unit/kafka/server/KafkaConfigTest.scala | 8 -------- .../group/GroupCoordinatorConfig.java | 6 +++--- .../kafka/server/config/ServerConfigs.java | 4 ++-- .../kafka/network/SocketServerConfigs.java | 4 ++-- .../kafka/server/config/KRaftConfigs.java | 4 ++-- .../server/metrics/ClientMetricsConfigs.java | 18 ++++++++--------- .../kafka/server/metrics/MetricConfigs.java | 4 ++-- .../storage/internals/log/LogConfig.java | 6 +++--- .../apache/kafka/streams/StreamsConfig.java | 6 +++--- .../kafka/tools/BrokerApiVersionsCommand.java | 2 +- 36 files changed, 123 insertions(+), 126 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 471d3916cfb55..699275466ac87 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 @@ -156,13 +156,13 @@ public class AdminClientConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC). define(BOOTSTRAP_CONTROLLERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.HIGH, BOOTSTRAP_CONTROLLERS_DOC) .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC) @@ -241,7 +241,7 @@ public class AdminClientConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, @@ -285,11 +285,11 @@ public class AdminClientConfig extends AbstractConfig { atLeast(0), Importance.LOW, METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, + ConfigDef.ValidList.anyValues(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 ef2f7d0e9d7d8..f67d8b3024aef 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 @@ -416,7 +416,7 @@ public class ConsumerConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, @@ -446,7 +446,7 @@ public class ConsumerConfig extends AbstractConfig { .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.LIST, List.of(RangeAssignor.class, CooperativeStickyAssignor.class), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.MEDIUM, PARTITION_ASSIGNMENT_STRATEGY_DOC) .define(METADATA_MAX_AGE_CONFIG, @@ -573,7 +573,7 @@ public class ConsumerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(KEY_DESERIALIZER_CLASS_CONFIG, @@ -615,7 +615,7 @@ public class ConsumerConfig extends AbstractConfig { .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(MAX_POLL_RECORDS_CONFIG, @@ -704,7 +704,7 @@ public class ConsumerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } 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 313648497bab1..e5bd526b17bb0 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 @@ -376,7 +376,7 @@ public class ProducerConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, @@ -467,7 +467,7 @@ public class ProducerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, @@ -507,7 +507,7 @@ public class ProducerConfig extends AbstractConfig { .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, @@ -561,7 +561,7 @@ public class ProducerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index e271cd99c4cd0..2daf4644ebe3a 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -65,7 +65,7 @@ public class AbstractConfig { public static final String AUTOMATIC_CONFIG_PROVIDERS_PROPERTY = "org.apache.kafka.automatic.config.providers"; public static final String CONFIG_PROVIDERS_CONFIG = "config.providers"; - public static final String CONFIG_PROVIDERS_DOC = + public static final String CONFIG_PROVIDERS_DOC = "Comma-separated alias names for classes implementing the ConfigProvider interface. " + "This enables loading configuration data (such as passwords, API keys, and other credentials) from external " + "sources. For example, see
Configuration Providers."; @@ -115,10 +115,10 @@ public class AbstractConfig { public AbstractConfig(ConfigDef definition, Map originals, Map configProviderProps, boolean doLog) { Map originalMap = preProcessParsedConfig(Collections.unmodifiableMap(Utils.castToStringObjectMap(originals))); this.originals = resolveConfigVariables(configProviderProps, originalMap); - this.values = definition.parse(this.originals); + this.values = definition.parse(this.originals, allowDuplicateValueInList()); Map configUpdates = postProcessParsedConfig(Collections.unmodifiableMap(this.values)); this.values.putAll(configUpdates); - definition.parse(this.values); + definition.parse(this.values, allowDuplicateValueInList()); this.definition = definition; if (doLog) logAll(); @@ -316,18 +316,22 @@ public Map valuesWithPrefixOverride(String prefix) { String keyWithNoPrefix = entry.getKey().substring(prefix.length()); ConfigDef.ConfigKey configKey = definition.configKeys().get(keyWithNoPrefix); if (configKey != null) - result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true)); + result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); else { String keyWithNoSecondaryPrefix = keyWithNoPrefix.substring(keyWithNoPrefix.indexOf('.') + 1); configKey = definition.configKeys().get(keyWithNoSecondaryPrefix); if (configKey != null) - result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true)); + result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); } } } return result; } + protected boolean allowDuplicateValueInList() { + return false; + } + /** * If at least one key with {@code prefix} exists, all prefixed values will be parsed and put into map. * If no value with {@code prefix} exists all unprefixed values will be returned. @@ -346,7 +350,7 @@ public Map valuesWithPrefixAllOrNothing(String prefix) { for (Map.Entry entry : withPrefix.entrySet()) { ConfigDef.ConfigKey configKey = definition.configKeys().get(entry.getKey()); if (configKey != null) - result.put(entry.getKey(), definition.parseValue(configKey, entry.getValue(), true)); + result.put(entry.getKey(), definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); } return result; 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 5ae8204ba42f8..73ad2922e1271 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 @@ -512,6 +512,10 @@ public ConfigDef withClientSaslSupport() { * the appropriate type (int, string, etc). */ public Map parse(Map props) { + return parse(props, false); + } + + Map parse(Map props, boolean allowDuplicateValueInList) { // Check all configurations are defined List undefinedConfigKeys = undefinedDependentConfigs(); if (!undefinedConfigKeys.isEmpty()) { @@ -521,11 +525,11 @@ public Map parse(Map props) { // parse all known keys Map values = new HashMap<>(); for (ConfigKey key : configKeys.values()) - values.put(key.name, parseValue(key, props.get(key.name), props.containsKey(key.name))); + values.put(key.name, parseValue(key, props.get(key.name), props.containsKey(key.name), allowDuplicateValueInList)); return values; } - Object parseValue(ConfigKey key, Object value, boolean isSet) { + Object parseValue(ConfigKey key, Object value, boolean isSet, boolean allowDuplicateValueInList) { Object parsedValue; if (isSet) { parsedValue = parseType(key.name, value, key.type); @@ -536,6 +540,9 @@ Object parseValue(ConfigKey key, Object value, boolean isSet) { // otherwise assign setting its default value parsedValue = key.defaultValue; } + if (!allowDuplicateValueInList && parsedValue instanceof List) { + parsedValue = ((List) parsedValue).stream().distinct().collect(Collectors.toList()); + } if (key.validator != null) { key.validator.ensureValid(key.name, parsedValue); } @@ -1015,7 +1022,7 @@ private ValidList(List validStrings, boolean isEmptyAllowed, boolean isN this.isNullAllowed = isNullAllowed; } - public static ValidList anyNonDuplicateValues(boolean isEmptyAllowed, boolean isNullAllowed) { + public static ValidList anyValues(boolean isEmptyAllowed, boolean isNullAllowed) { return new ValidList(List.of(), isEmptyAllowed, isNullAllowed); } @@ -1046,11 +1053,6 @@ public void ensureValid(final String name, final Object value) { throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); } - if (Set.copyOf(values).size() != values.size()) { - System.out.println("Configuration '" + name + "' has duplicate values: " + values + - "this will be disallowed in Kafka5.0."); - } - validateIndividualValues(name, values); } 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 01f7ad1f92718..5e552762e95b0 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 @@ -409,7 +409,7 @@ public static void addClientSaslSupport(ConfigDef config) { .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, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyValues(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 7675f75a9ab71..ccfb65f27e97e 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 @@ -50,8 +50,8 @@ 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. 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. " + + "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"; @@ -126,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, 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_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyValues(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.anyValues(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/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index 48f3948ef9d1f..1e78a45cc361e 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, ConfigDef.ValidList.anyNonDuplicateValues(true, false), MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyValues(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) 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 c75e08779cc53..08fc6ed5b7a1a 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 @@ -760,7 +760,7 @@ public void testListSizeValidatorToString() { @Test public void testListValidatorAnyNonDuplicateValues() { - ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyListAndNull = ConfigDef.ValidList.anyNonDuplicateValues(true, true); + ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyListAndNull = ConfigDef.ValidList.anyValues(true, true); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "b", "c"))); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of())); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", null)); @@ -769,7 +769,7 @@ public void testListValidatorAnyNonDuplicateValues() { ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); - ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyNonDuplicateValues(false, true); + ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyValues(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())); @@ -779,7 +779,7 @@ public void testListValidatorAnyNonDuplicateValues() { 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); + ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyList = ConfigDef.ValidList.anyValues(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)); @@ -789,7 +789,7 @@ public void testListValidatorAnyNonDuplicateValues() { ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception8.getMessage()); - ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyNonDuplicateValues(false, false); + ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyValues(false, false); assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "b", "c"))); ConfigException exception9 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); assertEquals("Configuration 'test.config' values must not be null.", exception9.getMessage()); 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 cb42f5fe654ba..3327644b0fbfc 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 @@ -119,7 +119,7 @@ private Map clientConfig(String prefix) { .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) // security support @@ -136,7 +136,7 @@ private Map clientConfig(String prefix) { .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( 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 ded82a8571c7f..d8eb857ebde3c 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,13 +65,13 @@ static class GroupFilterConfig extends AbstractConfig { .define(GROUPS_INCLUDE_CONFIG, Type.LIST, GROUPS_INCLUDE_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.HIGH, GROUPS_INCLUDE_DOC) .define(GROUPS_EXCLUDE_CONFIG, Type.LIST, GROUPS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(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 b7625da619ddf..593de7b7c8b36 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,14 +194,14 @@ private static ConfigDef defineCheckpointConfig(ConfigDef baseConfig) { GROUPS, ConfigDef.Type.LIST, GROUPS_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.HIGH, GROUPS_DOC) .define( GROUPS_EXCLUDE, ConfigDef.Type.LIST, GROUPS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(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 3d2cfda6dcc9a..df81c439a71e7 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 @@ -50,7 +50,7 @@ String entityLabel() { TASK_CONSUMER_GROUPS, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(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 9baf7c1f35cb5..69d469d953000 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,7 +311,7 @@ String entityLabel() { CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, ConfigDef.Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define( @@ -324,8 +324,8 @@ String entityLabel() { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, + ConfigDef.ValidList.anyValues(true, false), + ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) .withClientSslSupport() .withClientSaslSupport(); 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 33fc2641a394c..1d95ff66b0823 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 @@ -169,11 +169,11 @@ Map clusterProps(String cluster) { props.putIfAbsent(k, v); } } - + return props; } - // loads worker configs based on properties of the form x.y.z and cluster.x.y.z + // loads worker configs based on properties of the form x.y.z and cluster.x.y.z public Map workerConfig(SourceAndTarget sourceAndTarget) { Map props = new HashMap<>(); props.putAll(clusterProps(sourceAndTarget.target())); @@ -184,8 +184,8 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { props.putAll(stringsWithPrefix("offset.storage")); props.putAll(stringsWithPrefix("config.storage")); props.putAll(stringsWithPrefix("status.storage")); - props.putAll(stringsWithPrefix("key.converter")); - props.putAll(stringsWithPrefix("value.converter")); + props.putAll(stringsWithPrefix("key.converter")); + props.putAll(stringsWithPrefix("value.converter")); props.putAll(stringsWithPrefix("header.converter")); props.putAll(stringsWithPrefix("task")); props.putAll(stringsWithPrefix("worker")); @@ -204,8 +204,8 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { + sourceAndTarget.source() + ".internal"); props.putIfAbsent(DistributedConfig.CONFIG_TOPIC_CONFIG, "mm2-configs." + sourceAndTarget.source() + ".internal"); - props.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); - props.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); + props.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); + props.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); props.putIfAbsent(HEADER_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); return props; @@ -225,7 +225,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, props.putAll(rawProperties); props.keySet().retainAll(allConfigNames()); - + props.putAll(stringsWithPrefix(CONFIG_PROVIDERS_CONFIG)); props.putAll(stringsWithPrefix("replication.policy")); @@ -257,7 +257,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, List configProviders() { return getList(CONFIG_PROVIDERS_CONFIG); - } + } Map transform(Map props) { // transform worker config according to config.providers @@ -279,9 +279,9 @@ Map transform(Map props) { private static ConfigDef config() { ConfigDef result = new ConfigDef() - .define(CLUSTERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, CLUSTERS_DOC) + .define(CLUSTERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(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(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), ConfigDef.ValidList.anyValues(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 2ec663ad2fc32..db7824fdfe57b 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,21 +209,21 @@ private static ConfigDef defineSourceConfig(ConfigDef baseConfig) { TOPICS, ConfigDef.Type.LIST, TOPICS_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC) .define( TOPICS_EXCLUDE, ConfigDef.Type.LIST, TOPICS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(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.ValidList.anyValues(true, false), ConfigDef.Importance.HIGH, CONFIG_PROPERTIES_EXCLUDE_DOC) .define( @@ -323,7 +323,7 @@ private static ConfigDef defineSourceConfig(ConfigDef baseConfig) { protected static final ConfigDef CONNECTOR_CONFIG_DEF = defineSourceConfig(new ConfigDef(BASE_CONNECTOR_CONFIG_DEF)); - public static void main(String[] args) { + public static void main(String[] args) { System.out.println(defineSourceConfig(new ConfigDef()).toHtml(4, config -> "mirror_source_" + config)); } } 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 aa5d300c00ab9..c9914cb2578af 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 @@ -49,13 +49,13 @@ MirrorSourceMetrics metrics() { String entityLabel() { return super.entityLabel() + "-" + (getInt(TASK_INDEX) == null ? "?" : getInt(TASK_INDEX)); } - + protected static final ConfigDef TASK_CONFIG_DEF = new ConfigDef(CONNECTOR_CONFIG_DEF) .define( TASK_TOPIC_PARTITIONS, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), ConfigDef.Importance.LOW, TASK_TOPIC_PARTITIONS_DOC) .define(TASK_INDEX, 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 43da717a78cf4..2c93844050cf3 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 @@ -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.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) + .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.ValidList.anyValues(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 35326c03ae3db..04749711a964a 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 @@ -200,7 +200,7 @@ public class WorkerConfig extends AbstractConfig { protected static ConfigDef baseConfigDef() { ConfigDef result = new ConfigDef() .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, @@ -227,7 +227,7 @@ protected static ConfigDef baseConfigDef() { .define(PLUGIN_PATH_CONFIG, Type.LIST, null, - ConfigDef.ValidList.anyNonDuplicateValues(false, true), + ConfigDef.ValidList.anyValues(false, true), Importance.LOW, PLUGIN_PATH_DOC) .define(PLUGIN_DISCOVERY_CONFIG, @@ -249,7 +249,7 @@ protected static ConfigDef baseConfigDef() { CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS, @@ -260,7 +260,7 @@ protected static ConfigDef baseConfigDef() { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(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) 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 1a08a7eb123d4..6d9a500c7e220 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 @@ -143,12 +143,12 @@ public static void addPublicConfig(ConfigDef configDef) { .define(REST_EXTENSION_CLASSES_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC) .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - ConfigDef.ValidList.anyNonDuplicateValues(true, true), + ConfigDef.ValidList.anyValues(true, true), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } 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 cd87c33a5095e..157b713e2ea2d 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 @@ -40,8 +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, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyValues(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 7e8f6700bf634..6a5c583594a17 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,16 +55,16 @@ interface ConfigName { } public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(ConfigName.EXCLUDE, - ConfigDef.Type.LIST, - List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + .define(ConfigName.EXCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyValues(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.ValidList.anyNonDuplicateValues(true, false), + .define(ConfigName.INCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyValues(true, false), ConfigDef.Importance.MEDIUM, "Fields to include. If specified, only these fields will be used.") .define(ConfigName.RENAMES, ConfigDef.Type.LIST, List.of(), 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 19c299e6867e6..f088ed5eb34ff 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 @@ -45,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, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(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/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java index 7629922589ef2..40b50dd09a5b5 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java @@ -30,4 +30,8 @@ public SimpleConfig(ConfigDef configDef, Map originals) { super(configDef, originals, false); } + @Override + protected boolean allowDuplicateValueInList() { + return true; + } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index f9b8e05ef9ef9..a7317a20e896d 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -221,12 +221,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"Configuration '${KRaftConfigs.PROCESS_ROLES_CONFIG}' values must not be duplicated.") - } - distinctRoles + roles.toSet } def isKRaftCombinedMode: Boolean = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 157c46ae3e877..cc72b8a3debcf 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -220,10 +220,6 @@ class KafkaConfigTest { props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "HOST://localhost:9091,LB://localhost:9092") props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "HOST://localhost:9091,LB://localhost:9091") KafkaConfig.fromProps(props) - - // 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") } @Test @@ -247,10 +243,6 @@ class KafkaConfigTest { caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) 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")) - 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)) assertTrue(caught.getMessage.contains("Each listener must have a different port")) 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 e8a2f49663955..28c724a7aa343 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 @@ -297,7 +297,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, + .define(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, 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) @@ -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, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) + .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyValues(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, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, SHARE_GROUP_ASSIGNORS_DOC) + .define(SHARE_GROUP_ASSIGNORS_CONFIG, LIST, SHARE_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyValues(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 e40c5c8e6d36f..99404feb86b8b 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 @@ -134,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.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyValues(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, LIST, null, ConfigDef.ValidList.anyNonDuplicateValues(true, true), HIGH, EARLY_START_LISTENERS_DOC) + .define(EARLY_START_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyValues(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/src/main/java/org/apache/kafka/network/SocketServerConfigs.java b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java index d9d2988d0edd7..935119220b5c7 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -153,8 +153,8 @@ public class SocketServerConfigs { 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, 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(LISTENERS_CONFIG, LIST, LISTENERS_DEFAULT, ConfigDef.ValidList.anyValues(false, false), HIGH, LISTENERS_DOC) + .define(ADVERTISED_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyValues(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) 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 365f0937cd805..15b9355c0f62a 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 @@ -31,7 +31,7 @@ public class KRaftConfigs { /** KRaft mode configs */ public static final String PROCESS_ROLES_CONFIG = "process.roles"; public static final String PROCESS_ROLES_DOC = "The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. "; - + public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG = "initial.broker.registration.timeout.ms"; public static final int INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT = 60000; public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC = "When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process."; @@ -75,7 +75,7 @@ public class KRaftConfigs { .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, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, CONTROLLER_LISTENER_NAMES_DOC) + .define(CONTROLLER_LISTENER_NAMES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(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 b7a66df7b1620..0a3c788fb3e90 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,18 +102,18 @@ public class ClientMetricsConfigs extends AbstractConfig { ); private static final ConfigDef CONFIG = new ConfigDef() - .define(METRICS_CONFIG, - Type.LIST, - METRICS_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), - Importance.MEDIUM, + .define(METRICS_CONFIG, + Type.LIST, + METRICS_DEFAULT, + ConfigDef.ValidList.anyValues(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, + .define(MATCH_CONFIG, + Type.LIST, MATCH_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), - Importance.MEDIUM, + ConfigDef.ValidList.anyValues(true, false), + Importance.MEDIUM, "Client match criteria"); public ClientMetricsConfigs(Properties props) { 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 a974c36bed5b4..a7480d29e5caf 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 @@ -71,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, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, METRIC_REPORTER_CLASSES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, LIST, METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyValues(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, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, KAFKA_METRICS_REPORTER_CLASSES_DOC) + .define(KAFKA_METRICS_REPORTER_CLASSES_CONFIG, LIST, KAFKA_METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyValues(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/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 5d0bec2745304..723c838f9abef 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 Optional serverConfigName(String configName) { 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, 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_DIR_CONFIG, LIST, ServerLogConfigs.LOG_DIR_DEFAULT, ConfigDef.ValidList.anyValues(false, false), HIGH, ServerLogConfigs.LOG_DIR_DOC) + .define(ServerLogConfigs.LOG_DIRS_CONFIG, LIST, null, ConfigDef.ValidList.anyValues(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) @@ -562,7 +562,7 @@ public static void validateRemoteStorageOnlyIfSystemEnabled(Map props, boo 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 (!policySet.isEmpty() && (policySet.size() != 1 || !Set.of(TopicConfig.CLEANUP_POLICY_DELETE).equals(policySet))) { + 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/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 24701a7de9678..65217bb8510b5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -848,7 +848,7 @@ public class StreamsConfig extends AbstractConfig { .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.ValidList.anyValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, @@ -1052,7 +1052,7 @@ public class StreamsConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, @@ -1129,7 +1129,7 @@ public class StreamsConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.ValidList.anyValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, 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 9090fe46e4756..8dc8789eb9527 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 @@ protected static class AdminClient implements AutoCloseable { 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.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(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) From 8468ca75ee55bd7c43b4c3b002e612391184b635 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:19:27 +0800 Subject: [PATCH 06/16] addressed by comments --- docs/upgrade.html | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 06c12da18f5b6..a82e9343efba3 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -139,8 +139,6 @@
Notable changes in 4
  • 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.
  • -
  • Most LIST-type configurations no longer accept duplicate entries, except in cases where duplicates - are explicitly supported.
  • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined semantic meaning.
From 4909e8481129c03043b39944d434de1ca8dc44f2 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:29:04 +0800 Subject: [PATCH 07/16] addressed by comments --- .../clients/admin/AdminClientConfig.java | 8 +- .../clients/consumer/ConsumerConfig.java | 10 +-- .../clients/producer/ProducerConfig.java | 8 +- .../apache/kafka/common/config/ConfigDef.java | 6 +- .../kafka/common/config/SaslConfigs.java | 2 +- .../kafka/common/config/SslConfigs.java | 4 +- .../internals/BrokerSecurityConfigs.java | 2 +- .../kafka/common/config/ConfigDefTest.java | 80 +++++++++++-------- .../connect/mirror/MirrorClientConfig.java | 4 +- .../connect/mirror/DefaultGroupFilter.java | 4 +- .../mirror/MirrorCheckpointConfig.java | 4 +- .../mirror/MirrorCheckpointTaskConfig.java | 2 +- .../connect/mirror/MirrorConnectorConfig.java | 4 +- .../connect/mirror/MirrorMakerConfig.java | 4 +- .../connect/mirror/MirrorSourceConfig.java | 6 +- .../mirror/MirrorSourceTaskConfig.java | 2 +- .../connect/runtime/SinkConnectorConfig.java | 2 +- .../kafka/connect/runtime/WorkerConfig.java | 8 +- .../runtime/rest/RestServerConfig.java | 4 +- .../kafka/connect/transforms/DropHeaders.java | 2 +- .../kafka/connect/transforms/MaskField.java | 2 +- .../connect/transforms/ReplaceField.java | 4 +- .../kafka/connect/transforms/ValueToKey.java | 2 +- .../group/GroupCoordinatorConfig.java | 4 +- .../kafka/server/config/ServerConfigs.java | 4 +- .../kafka/network/SocketServerConfigs.java | 4 +- .../kafka/server/config/KRaftConfigs.java | 2 +- .../server/metrics/ClientMetricsConfigs.java | 4 +- .../kafka/server/metrics/MetricConfigs.java | 4 +- .../storage/internals/log/LogConfig.java | 5 +- .../apache/kafka/streams/StreamsConfig.java | 6 +- .../kafka/tools/BrokerApiVersionsCommand.java | 2 +- 32 files changed, 113 insertions(+), 96 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 699275466ac87..ada40ad7aaec2 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 @@ -156,13 +156,13 @@ public class AdminClientConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC). define(BOOTSTRAP_CONTROLLERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_CONTROLLERS_DOC) .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC) @@ -241,7 +241,7 @@ public class AdminClientConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, @@ -288,7 +288,7 @@ public class AdminClientConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + 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 f67d8b3024aef..ef2f7d0e9d7d8 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 @@ -416,7 +416,7 @@ public class ConsumerConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, @@ -446,7 +446,7 @@ public class ConsumerConfig extends AbstractConfig { .define(PARTITION_ASSIGNMENT_STRATEGY_CONFIG, Type.LIST, List.of(RangeAssignor.class, CooperativeStickyAssignor.class), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.MEDIUM, PARTITION_ASSIGNMENT_STRATEGY_DOC) .define(METADATA_MAX_AGE_CONFIG, @@ -573,7 +573,7 @@ public class ConsumerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(KEY_DESERIALIZER_CLASS_CONFIG, @@ -615,7 +615,7 @@ public class ConsumerConfig extends AbstractConfig { .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(MAX_POLL_RECORDS_CONFIG, @@ -704,7 +704,7 @@ public class ConsumerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } 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 e5bd526b17bb0..313648497bab1 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 @@ -376,7 +376,7 @@ public class ProducerConfig extends AbstractConfig { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, @@ -467,7 +467,7 @@ public class ProducerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, @@ -507,7 +507,7 @@ public class ProducerConfig extends AbstractConfig { .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, @@ -561,7 +561,7 @@ public class ProducerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + 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 73ad2922e1271..360572a025d41 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 @@ -1022,7 +1022,7 @@ private ValidList(List validStrings, boolean isEmptyAllowed, boolean isN this.isNullAllowed = isNullAllowed; } - public static ValidList anyValues(boolean isEmptyAllowed, boolean isNullAllowed) { + public static ValidList anyNonDuplicateValues(boolean isEmptyAllowed, boolean isNullAllowed) { return new ValidList(List.of(), isEmptyAllowed, isNullAllowed); } @@ -1053,6 +1053,10 @@ public void ensureValid(final String name, final Object value) { 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); } 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 5e552762e95b0..01f7ad1f92718 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 @@ -409,7 +409,7 @@ public static void addClientSaslSupport(ConfigDef config) { .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, List.of(), ConfigDef.ValidList.anyValues(true, false), 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 ccfb65f27e97e..9e613be38af86 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 @@ -126,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, List.of(), ConfigDef.ValidList.anyValues(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.anyValues(true, false), 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/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index 1e78a45cc361e..48f3948ef9d1f 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, ConfigDef.ValidList.anyValues(true, false), 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) 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 08fc6ed5b7a1a..34f53340b9bb6 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 @@ -760,45 +760,55 @@ public void testListSizeValidatorToString() { @Test public void testListValidatorAnyNonDuplicateValues() { - ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyListAndNull = ConfigDef.ValidList.anyValues(true, true); + ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyListAndNull = ConfigDef.ValidList.anyNonDuplicateValues(true, true); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "b", "c"))); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of())); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", null)); - ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "", "a"))); - assertEquals("Configuration 'test.config' values must not be empty.", exception1.getMessage()); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception3.getMessage()); - ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyValues(false, true); + 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 empty.", 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.anyValues(true, false); + ConfigException exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception5.getMessage()); + ConfigException exception6 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception6.getMessage()); + ConfigException exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception7.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 empty.", exception7.getMessage()); - ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); - assertEquals("Configuration 'test.config' values must not be empty.", exception8.getMessage()); + ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception8.getMessage()); + ConfigException exception9 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception9.getMessage()); + ConfigException exception10 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception10.getMessage()); + ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception11.getMessage()); + - ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyValues(false, false); + ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyNonDuplicateValues(false, false); assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "b", "c"))); - ConfigException exception9 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); - assertEquals("Configuration 'test.config' values must not be null.", exception9.getMessage()); - ConfigException exception10 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); - assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception10.getMessage()); - ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); - assertEquals("Configuration 'test.config' values must not be empty.", exception11.getMessage()); - ConfigException exception12 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); - assertEquals("Configuration 'test.config' values must not be empty.", exception12.getMessage()); + ConfigException exception12 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception12.getMessage()); + ConfigException exception13 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception13.getMessage()); + ConfigException exception14 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception14.getMessage()); + ConfigException exception15 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception15.getMessage()); + ConfigException exception16 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); + assertEquals("Configuration 'test.config' values must not be empty.", exception16.getMessage()); } @Test @@ -810,14 +820,18 @@ public void testListValidatorIn() { 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 exception3 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of())); - assertEquals("Configuration 'test.config' must not be empty. Valid values include: [a, b, c]", exception3.getMessage()); - ConfigException exception4 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", null)); - assertEquals("Configuration 'test.config' values must not be null.", exception4.getMessage()); - ConfigException exception5 = 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", exception5.getMessage()); + 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/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 3327644b0fbfc..cb42f5fe654ba 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 @@ -119,7 +119,7 @@ private Map clientConfig(String prefix) { .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) // security support @@ -136,7 +136,7 @@ private Map clientConfig(String prefix) { .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( 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 d8eb857ebde3c..ded82a8571c7f 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,13 +65,13 @@ static class GroupFilterConfig extends AbstractConfig { .define(GROUPS_INCLUDE_CONFIG, Type.LIST, GROUPS_INCLUDE_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, GROUPS_INCLUDE_DOC) .define(GROUPS_EXCLUDE_CONFIG, Type.LIST, GROUPS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + 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 593de7b7c8b36..b7625da619ddf 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,14 +194,14 @@ private static ConfigDef defineCheckpointConfig(ConfigDef baseConfig) { GROUPS, ConfigDef.Type.LIST, GROUPS_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, GROUPS_DOC) .define( GROUPS_EXCLUDE, ConfigDef.Type.LIST, GROUPS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + 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 df81c439a71e7..3d2cfda6dcc9a 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 @@ -50,7 +50,7 @@ String entityLabel() { TASK_CONSUMER_GROUPS, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + 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 69d469d953000..2bc0986597c84 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,7 +311,7 @@ String entityLabel() { CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, ConfigDef.Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define( @@ -324,7 +324,7 @@ String entityLabel() { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + 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 1d95ff66b0823..56b42a0bbe3bd 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 @@ Map transform(Map props) { private static ConfigDef config() { ConfigDef result = new ConfigDef() - .define(CLUSTERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(true, false), 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(), ConfigDef.ValidList.anyValues(true, false), 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 db7824fdfe57b..a2a500b653adc 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,21 +209,21 @@ private static ConfigDef defineSourceConfig(ConfigDef baseConfig) { TOPICS, ConfigDef.Type.LIST, TOPICS_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC) .define( TOPICS_EXCLUDE, ConfigDef.Type.LIST, TOPICS_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_EXCLUDE_DOC) .define( CONFIG_PROPERTIES_EXCLUDE, ConfigDef.Type.LIST, CONFIG_PROPERTIES_EXCLUDE_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + 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 c9914cb2578af..7aa76a37abb52 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 @@ -55,7 +55,7 @@ String entityLabel() { TASK_TOPIC_PARTITIONS, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.LOW, TASK_TOPIC_PARTITIONS_DOC) .define(TASK_INDEX, 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 2c93844050cf3..43da717a78cf4 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 @@ -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.ValidList.anyValues(true, false), 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 04749711a964a..35326c03ae3db 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 @@ -200,7 +200,7 @@ public class WorkerConfig extends AbstractConfig { protected static ConfigDef baseConfigDef() { ConfigDef result = new ConfigDef() .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, @@ -227,7 +227,7 @@ protected static ConfigDef baseConfigDef() { .define(PLUGIN_PATH_CONFIG, Type.LIST, null, - ConfigDef.ValidList.anyValues(false, true), + ConfigDef.ValidList.anyNonDuplicateValues(false, true), Importance.LOW, PLUGIN_PATH_DOC) .define(PLUGIN_DISCOVERY_CONFIG, @@ -249,7 +249,7 @@ protected static ConfigDef baseConfigDef() { CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS, @@ -260,7 +260,7 @@ protected static ConfigDef baseConfigDef() { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + 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) 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 6d9a500c7e220..1a08a7eb123d4 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 @@ -143,12 +143,12 @@ public static void addPublicConfig(ConfigDef configDef) { .define(REST_EXTENSION_CLASSES_CONFIG, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC) .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - ConfigDef.ValidList.anyValues(true, true), + ConfigDef.ValidList.anyNonDuplicateValues(true, true), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } 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 157b713e2ea2d..55fc45ec50cd4 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 @@ -41,7 +41,7 @@ public class DropHeaders> implements Transformation> implements Transform public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + 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 6a5c583594a17..bb35e1a32b58c 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 @@ -58,13 +58,13 @@ interface ConfigName { .define(ConfigName.EXCLUDE, ConfigDef.Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + 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.ValidList.anyValues(true, false), + 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(), 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 f088ed5eb34ff..19c299e6867e6 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 @@ -45,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, ConfigDef.ValidList.anyValues(false, false), 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/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 28c724a7aa343..4e0137db0d527 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 @@ -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, ConfigDef.ValidList.anyValues(false, false), 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, ConfigDef.ValidList.anyValues(false, false), 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 99404feb86b8b..e40c5c8e6d36f 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 @@ -134,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.ValidList.anyValues(true, false), 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, LIST, null, ConfigDef.ValidList.anyValues(true, true), 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/src/main/java/org/apache/kafka/network/SocketServerConfigs.java b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java index 935119220b5c7..d9d2988d0edd7 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -153,8 +153,8 @@ public class SocketServerConfigs { 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, LIST, LISTENERS_DEFAULT, ConfigDef.ValidList.anyValues(false, false), HIGH, LISTENERS_DOC) - .define(ADVERTISED_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyValues(false, true), HIGH, ADVERTISED_LISTENERS_DOC) + .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) 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 15b9355c0f62a..3f3aaa8e5101b 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 @@ -75,7 +75,7 @@ public class KRaftConfigs { .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, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(false, false), 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 0a3c788fb3e90..fd75b2c7b2e5f 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 @@ -105,14 +105,14 @@ public class ClientMetricsConfigs extends AbstractConfig { .define(METRICS_CONFIG, Type.LIST, METRICS_DEFAULT, - ConfigDef.ValidList.anyValues(true, false), + 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, - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.MEDIUM, "Client match criteria"); 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 a7480d29e5caf..a974c36bed5b4 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 @@ -71,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, ConfigDef.ValidList.anyValues(true, false), 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, ConfigDef.ValidList.anyValues(true, false), 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/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 723c838f9abef..a687f3c529e32 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 Optional serverConfigName(String configName) { 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, LIST, ServerLogConfigs.LOG_DIR_DEFAULT, ConfigDef.ValidList.anyValues(false, false), HIGH, ServerLogConfigs.LOG_DIR_DOC) - .define(ServerLogConfigs.LOG_DIRS_CONFIG, LIST, null, ConfigDef.ValidList.anyValues(false, true), 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,7 +561,6 @@ public static void validateRemoteStorageOnlyIfSystemEnabled(Map props, boo @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 (!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/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index b3a1e150343a4..0c37d8b2ce43b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -848,7 +848,7 @@ public class StreamsConfig extends AbstractConfig { .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyValues(false, false), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, @@ -1052,7 +1052,7 @@ public class StreamsConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, @@ -1129,7 +1129,7 @@ public class StreamsConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - ConfigDef.ValidList.anyValues(true, false), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, 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 8dc8789eb9527..9090fe46e4756 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 @@ protected static class AdminClient implements AutoCloseable { 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.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyValues(false, false), 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) From 93c860a89eac2da13d3a6538ebddd7a2a920de64 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:38:28 +0800 Subject: [PATCH 08/16] revert unused changed --- .../kafka/clients/admin/AdminClientConfig.java | 4 ++-- .../apache/kafka/common/config/SslConfigs.java | 4 ++-- .../connect/mirror/MirrorConnectorConfig.java | 2 +- .../kafka/connect/mirror/MirrorMakerConfig.java | 16 ++++++++-------- .../kafka/connect/mirror/MirrorSourceConfig.java | 2 +- .../connect/mirror/MirrorSourceTaskConfig.java | 2 +- .../kafka/connect/transforms/DropHeaders.java | 2 +- .../kafka/connect/transforms/MaskField.java | 2 +- .../kafka/connect/transforms/ReplaceField.java | 14 +++++++------- docs/upgrade.html | 3 +++ .../group/GroupCoordinatorConfig.java | 2 +- .../apache/kafka/server/config/KRaftConfigs.java | 2 +- .../server/metrics/ClientMetricsConfigs.java | 16 ++++++++-------- 13 files changed, 37 insertions(+), 34 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 ada40ad7aaec2..471d3916cfb55 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 @@ -285,11 +285,11 @@ public class AdminClientConfig extends AbstractConfig { atLeast(0), Importance.LOW, METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, + ConfigDef.Importance.LOW, CONFIG_PROVIDERS_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 9e613be38af86..7675f75a9ab71 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 @@ -50,8 +50,8 @@ 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. 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. " + + "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"; 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 2bc0986597c84..9baf7c1f35cb5 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 @@ -325,7 +325,7 @@ String entityLabel() { ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, + ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) .withClientSslSupport() .withClientSaslSupport(); 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 56b42a0bbe3bd..33fc2641a394c 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 @@ -169,11 +169,11 @@ Map clusterProps(String cluster) { props.putIfAbsent(k, v); } } - + return props; } - // loads worker configs based on properties of the form x.y.z and cluster.x.y.z + // loads worker configs based on properties of the form x.y.z and cluster.x.y.z public Map workerConfig(SourceAndTarget sourceAndTarget) { Map props = new HashMap<>(); props.putAll(clusterProps(sourceAndTarget.target())); @@ -184,8 +184,8 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { props.putAll(stringsWithPrefix("offset.storage")); props.putAll(stringsWithPrefix("config.storage")); props.putAll(stringsWithPrefix("status.storage")); - props.putAll(stringsWithPrefix("key.converter")); - props.putAll(stringsWithPrefix("value.converter")); + props.putAll(stringsWithPrefix("key.converter")); + props.putAll(stringsWithPrefix("value.converter")); props.putAll(stringsWithPrefix("header.converter")); props.putAll(stringsWithPrefix("task")); props.putAll(stringsWithPrefix("worker")); @@ -204,8 +204,8 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { + sourceAndTarget.source() + ".internal"); props.putIfAbsent(DistributedConfig.CONFIG_TOPIC_CONFIG, "mm2-configs." + sourceAndTarget.source() + ".internal"); - props.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); - props.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); + props.putIfAbsent(KEY_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); + props.putIfAbsent(VALUE_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); props.putIfAbsent(HEADER_CONVERTER_CLASS_CONFIG, BYTE_ARRAY_CONVERTER_CLASS); return props; @@ -225,7 +225,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, props.putAll(rawProperties); props.keySet().retainAll(allConfigNames()); - + props.putAll(stringsWithPrefix(CONFIG_PROVIDERS_CONFIG)); props.putAll(stringsWithPrefix("replication.policy")); @@ -257,7 +257,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, List configProviders() { return getList(CONFIG_PROVIDERS_CONFIG); - } + } Map transform(Map props) { // transform worker config according to config.providers 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 a2a500b653adc..2ec663ad2fc32 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 @@ -323,7 +323,7 @@ private static ConfigDef defineSourceConfig(ConfigDef baseConfig) { protected static final ConfigDef CONNECTOR_CONFIG_DEF = defineSourceConfig(new ConfigDef(BASE_CONNECTOR_CONFIG_DEF)); - public static void main(String[] args) { + public static void main(String[] args) { System.out.println(defineSourceConfig(new ConfigDef()).toHtml(4, config -> "mirror_source_" + config)); } } 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 7aa76a37abb52..aa5d300c00ab9 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 @@ -49,7 +49,7 @@ MirrorSourceMetrics metrics() { String entityLabel() { return super.entityLabel() + "-" + (getInt(TASK_INDEX) == null ? "?" : getInt(TASK_INDEX)); } - + protected static final ConfigDef TASK_CONFIG_DEF = new ConfigDef(CONNECTOR_CONFIG_DEF) .define( TASK_TOPIC_PARTITIONS, 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 55fc45ec50cd4..cd87c33a5095e 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 @@ -40,7 +40,7 @@ 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, + .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(), 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 bb35e1a32b58c..7e8f6700bf634 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,15 +55,15 @@ interface ConfigName { } public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(ConfigName.EXCLUDE, - ConfigDef.Type.LIST, - List.of(), - ConfigDef.ValidList.anyNonDuplicateValues(true, false), + .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(), + .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.") diff --git a/docs/upgrade.html b/docs/upgrade.html index a82e9343efba3..041928c33884e 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -139,6 +139,9 @@
Notable changes in 4
  • 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.
  • +
  • Most LIST-type configurations no longer accept duplicate entries, except in cases where duplicates + are explicitly supported. However, if users configure duplicate entries, the internal deduplication + logic will still handle them
  • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined semantic meaning.
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 4e0137db0d527..e8a2f49663955 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 @@ -297,7 +297,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, + .define(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, 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) 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 3f3aaa8e5101b..365f0937cd805 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 @@ -31,7 +31,7 @@ public class KRaftConfigs { /** KRaft mode configs */ public static final String PROCESS_ROLES_CONFIG = "process.roles"; public static final String PROCESS_ROLES_DOC = "The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. "; - + public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG = "initial.broker.registration.timeout.ms"; public static final int INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT = 60000; public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC = "When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process."; 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 fd75b2c7b2e5f..b7a66df7b1620 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,18 +102,18 @@ public class ClientMetricsConfigs extends AbstractConfig { ); private static final ConfigDef CONFIG = new ConfigDef() - .define(METRICS_CONFIG, - Type.LIST, - METRICS_DEFAULT, - ConfigDef.ValidList.anyNonDuplicateValues(true, false), - Importance.MEDIUM, + .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, + .define(MATCH_CONFIG, + Type.LIST, MATCH_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), - Importance.MEDIUM, + Importance.MEDIUM, "Client match criteria"); public ClientMetricsConfigs(Properties props) { From c2831ca160008de289f14e011044a45e5f5f367e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:53:18 +0800 Subject: [PATCH 09/16] add new warning message --- .../java/org/apache/kafka/common/config/ConfigDef.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) 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 360572a025d41..e96bc64f6d3a9 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 @@ -18,6 +18,8 @@ import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.ArrayList; import java.util.Arrays; @@ -81,7 +83,7 @@ public class ConfigDef { private static final Pattern COMMA_WITH_WHITESPACE = Pattern.compile("\\s*,\\s*"); - + private static final Logger LOGGER = LoggerFactory.getLogger(ConfigDef.class); /** * A unique Java object which represents the lack of a default value. */ @@ -541,7 +543,10 @@ Object parseValue(ConfigKey key, Object value, boolean isSet, boolean allowDupli parsedValue = key.defaultValue; } if (!allowDuplicateValueInList && parsedValue instanceof List) { - parsedValue = ((List) parsedValue).stream().distinct().collect(Collectors.toList()); + List originalListValue = (List) parsedValue; + parsedValue = originalListValue.stream().distinct().collect(Collectors.toList()); + LOGGER.warn("Duplicate configuration \"{}\" values are found. Duplicates will be removed. The original value " + + "is: {}, the updated value is: {}", key.name, originalListValue, parsedValue); } if (key.validator != null) { key.validator.ensureValid(key.name, parsedValue); From f0f103e2852f2a5983f4e255bc50241f404aaf19 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 11 Nov 2025 21:59:59 +0800 Subject: [PATCH 10/16] format code --- .../src/main/java/org/apache/kafka/common/config/ConfigDef.java | 1 + 1 file changed, 1 insertion(+) 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 e96bc64f6d3a9..a050cbaacf002 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 @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.utils.Utils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; From e1c4af346ef840f9cb1ebb9360e4ed4a93f45c1d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 12 Nov 2025 09:14:05 +0800 Subject: [PATCH 11/16] fix fail test --- checkstyle/suppressions.xml | 1 + .../common/config/AbstractConfigTest.java | 45 ++++++++++++++++--- .../kafka/common/config/ConfigDefTest.java | 10 ++--- 3 files changed, 46 insertions(+), 10 deletions(-) diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 85b16fd65d423..36ac72dfec60b 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -133,6 +133,7 @@ + props = new HashMap<>(); - String threeConsumerInterceptors = MockConsumerInterceptor.class.getName() + ", " - + MockConsumerInterceptor.class.getName() + ", " + String threeConsumerInterceptors = CloseInterceptor.class.getName() + ", " + MockConsumerInterceptor.class.getName(); props.put(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, threeConsumerInterceptors); props.put("client.id", "test"); TestConfig testConfig = new TestConfig(props); - MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(3); + MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(1); assertThrows( Exception.class, () -> testConfig.getConfiguredInstances(TestConfig.METRIC_REPORTER_CLASSES_CONFIG, Object.class) ); - 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(); } } @@ -746,4 +752,33 @@ public void configure(Map configs) { configs.get(EXTRA_CONFIG); } } + + 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/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 34f53340b9bb6..1011fc21ac080 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 @@ -768,7 +768,7 @@ public void testListValidatorAnyNonDuplicateValues() { assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); - ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "", "a"))); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyListAndNull.ensureValid("test.config", List.of("a", "", "b"))); assertEquals("Configuration 'test.config' values must not be empty.", exception3.getMessage()); ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyNonDuplicateValues(false, true); @@ -780,7 +780,7 @@ public void testListValidatorAnyNonDuplicateValues() { assertEquals("Configuration 'test.config' values must not be duplicated.", exception5.getMessage()); ConfigException exception6 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception6.getMessage()); - ConfigException exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "", "a"))); + ConfigException exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "", "b"))); assertEquals("Configuration 'test.config' values must not be empty.", exception7.getMessage()); @@ -793,7 +793,7 @@ public void testListValidatorAnyNonDuplicateValues() { assertEquals("Configuration 'test.config' values must not be duplicated.", exception9.getMessage()); ConfigException exception10 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception10.getMessage()); - ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "", "a"))); + ConfigException exception11 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "", "b"))); assertEquals("Configuration 'test.config' values must not be empty.", exception11.getMessage()); @@ -803,11 +803,11 @@ public void testListValidatorAnyNonDuplicateValues() { assertEquals("Configuration 'test.config' values must not be null.", exception12.getMessage()); ConfigException exception13 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception13.getMessage()); - ConfigException exception14 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); + ConfigException exception14 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "b"))); assertEquals("Configuration 'test.config' values must not be empty.", exception14.getMessage()); ConfigException exception15 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); assertEquals("Configuration 'test.config' values must not be empty.", exception15.getMessage()); - ConfigException exception16 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "a"))); + ConfigException exception16 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "", "b"))); assertEquals("Configuration 'test.config' values must not be empty.", exception16.getMessage()); } From 3427ebb56da2e3896dd48963082aa200af6bb6f7 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 12 Nov 2025 20:50:28 +0800 Subject: [PATCH 12/16] add AllowlistConnectorClientConfigOverridePolicy for validator --- .../AllowlistConnectorClientConfigOverridePolicy.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AllowlistConnectorClientConfigOverridePolicy.java b/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AllowlistConnectorClientConfigOverridePolicy.java index 80d89db73e8cd..13a6b48f4819f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AllowlistConnectorClientConfigOverridePolicy.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/connector/policy/AllowlistConnectorClientConfigOverridePolicy.java @@ -40,7 +40,13 @@ public class AllowlistConnectorClientConfigOverridePolicy extends AbstractConnec private static final String ALLOWLIST_CONFIG_DOC = "List of client configurations that can be overridden by " + "connectors. If empty, connectors can't override any client configurations."; private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(ALLOWLIST_CONFIG, ConfigDef.Type.LIST, ALLOWLIST_CONFIG_DEFAULT, ConfigDef.Importance.MEDIUM, ALLOWLIST_CONFIG_DOC); + .define( + ALLOWLIST_CONFIG, + ConfigDef.Type.LIST, + ALLOWLIST_CONFIG_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.MEDIUM, ALLOWLIST_CONFIG_DOC + ); private List allowlist = ALLOWLIST_CONFIG_DEFAULT; From cca5d094b722775926285a2f6755478cf15c07d9 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 13 Nov 2025 08:42:13 +0800 Subject: [PATCH 13/16] updated the warning condition --- .../main/java/org/apache/kafka/common/config/ConfigDef.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 a050cbaacf002..2aff75272a0c8 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 @@ -546,8 +546,10 @@ Object parseValue(ConfigKey key, Object value, boolean isSet, boolean allowDupli if (!allowDuplicateValueInList && parsedValue instanceof List) { List originalListValue = (List) parsedValue; parsedValue = originalListValue.stream().distinct().collect(Collectors.toList()); - LOGGER.warn("Duplicate configuration \"{}\" values are found. Duplicates will be removed. The original value " + - "is: {}, the updated value is: {}", key.name, originalListValue, parsedValue); + if (originalListValue.size() != ((List) parsedValue).size()) { + LOGGER.warn("Duplicate configuration \"{}\" values are found. Duplicates will be removed. The original value " + + "is: {}, the updated value is: {}", key.name, originalListValue, parsedValue); + } } if (key.validator != null) { key.validator.ensureValid(key.name, parsedValue); From 2632c640e266aad8e5064601182a9e6c2dbbcef3 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 13 Nov 2025 09:36:00 +0800 Subject: [PATCH 14/16] addressed by comments --- .../kafka/common/config/AbstractConfig.java | 16 ++++++---------- .../apache/kafka/common/config/ConfigDef.java | 13 +++++-------- .../connect/transforms/util/SimpleConfig.java | 4 ---- 3 files changed, 11 insertions(+), 22 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 2daf4644ebe3a..e271cd99c4cd0 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -65,7 +65,7 @@ public class AbstractConfig { public static final String AUTOMATIC_CONFIG_PROVIDERS_PROPERTY = "org.apache.kafka.automatic.config.providers"; public static final String CONFIG_PROVIDERS_CONFIG = "config.providers"; - public static final String CONFIG_PROVIDERS_DOC = + public static final String CONFIG_PROVIDERS_DOC = "Comma-separated alias names for classes implementing the ConfigProvider interface. " + "This enables loading configuration data (such as passwords, API keys, and other credentials) from external " + "sources. For example, see
Configuration Providers."; @@ -115,10 +115,10 @@ public class AbstractConfig { public AbstractConfig(ConfigDef definition, Map originals, Map configProviderProps, boolean doLog) { Map originalMap = preProcessParsedConfig(Collections.unmodifiableMap(Utils.castToStringObjectMap(originals))); this.originals = resolveConfigVariables(configProviderProps, originalMap); - this.values = definition.parse(this.originals, allowDuplicateValueInList()); + this.values = definition.parse(this.originals); Map configUpdates = postProcessParsedConfig(Collections.unmodifiableMap(this.values)); this.values.putAll(configUpdates); - definition.parse(this.values, allowDuplicateValueInList()); + definition.parse(this.values); this.definition = definition; if (doLog) logAll(); @@ -316,22 +316,18 @@ public Map valuesWithPrefixOverride(String prefix) { String keyWithNoPrefix = entry.getKey().substring(prefix.length()); ConfigDef.ConfigKey configKey = definition.configKeys().get(keyWithNoPrefix); if (configKey != null) - result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); + result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true)); else { String keyWithNoSecondaryPrefix = keyWithNoPrefix.substring(keyWithNoPrefix.indexOf('.') + 1); configKey = definition.configKeys().get(keyWithNoSecondaryPrefix); if (configKey != null) - result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); + result.put(keyWithNoPrefix, definition.parseValue(configKey, entry.getValue(), true)); } } } return result; } - protected boolean allowDuplicateValueInList() { - return false; - } - /** * If at least one key with {@code prefix} exists, all prefixed values will be parsed and put into map. * If no value with {@code prefix} exists all unprefixed values will be returned. @@ -350,7 +346,7 @@ public Map valuesWithPrefixAllOrNothing(String prefix) { for (Map.Entry entry : withPrefix.entrySet()) { ConfigDef.ConfigKey configKey = definition.configKeys().get(entry.getKey()); if (configKey != null) - result.put(entry.getKey(), definition.parseValue(configKey, entry.getValue(), true, allowDuplicateValueInList())); + result.put(entry.getKey(), definition.parseValue(configKey, entry.getValue(), true)); } return result; 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 2aff75272a0c8..ed08928b8a6be 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 @@ -515,10 +515,6 @@ public ConfigDef withClientSaslSupport() { * the appropriate type (int, string, etc). */ public Map parse(Map props) { - return parse(props, false); - } - - Map parse(Map props, boolean allowDuplicateValueInList) { // Check all configurations are defined List undefinedConfigKeys = undefinedDependentConfigs(); if (!undefinedConfigKeys.isEmpty()) { @@ -528,11 +524,11 @@ Map parse(Map props, boolean allowDuplicateValueInList) { // parse all known keys Map values = new HashMap<>(); for (ConfigKey key : configKeys.values()) - values.put(key.name, parseValue(key, props.get(key.name), props.containsKey(key.name), allowDuplicateValueInList)); + values.put(key.name, parseValue(key, props.get(key.name), props.containsKey(key.name))); return values; } - Object parseValue(ConfigKey key, Object value, boolean isSet, boolean allowDuplicateValueInList) { + Object parseValue(ConfigKey key, Object value, boolean isSet) { Object parsedValue; if (isSet) { parsedValue = parseType(key.name, value, key.type); @@ -543,7 +539,7 @@ Object parseValue(ConfigKey key, Object value, boolean isSet, boolean allowDupli // otherwise assign setting its default value parsedValue = key.defaultValue; } - if (!allowDuplicateValueInList && parsedValue instanceof List) { + if (key.validator instanceof ValidList && parsedValue instanceof List) { List originalListValue = (List) parsedValue; parsedValue = originalListValue.stream().distinct().collect(Collectors.toList()); if (originalListValue.size() != ((List) parsedValue).size()) { @@ -1062,7 +1058,8 @@ public void ensureValid(final String name, final Object value) { } if (Set.copyOf(values).size() != values.size()) { - throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); + System.out.println("Configuration '" + name + "' has duplicate values: " + values + + "this will be disallowed in Kafka5.0."); } validateIndividualValues(name, values); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java index 40b50dd09a5b5..7629922589ef2 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SimpleConfig.java @@ -30,8 +30,4 @@ public SimpleConfig(ConfigDef configDef, Map originals) { super(configDef, originals, false); } - @Override - protected boolean allowDuplicateValueInList() { - return true; - } } From 835a119e1d9ed8f5c5b3c343e5d5de49c48de205 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 13 Nov 2025 10:05:11 +0800 Subject: [PATCH 15/16] addressed by comments --- .../org/apache/kafka/common/config/ConfigDef.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) 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 ed08928b8a6be..3974d733adeac 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 @@ -1058,8 +1058,7 @@ public void ensureValid(final String name, final Object value) { } if (Set.copyOf(values).size() != values.size()) { - System.out.println("Configuration '" + name + "' has duplicate values: " + values + - "this will be disallowed in Kafka5.0."); + throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); } validateIndividualValues(name, values); @@ -1082,8 +1081,15 @@ private void validateIndividualValues(String name, List values) { } public String toString() { - return validString + (isEmptyAllowed ? " (empty config allowed)" : " (empty not allowed)") + - (isNullAllowed ? " (null config allowed)" : " (null not allowed)"); + String base = validString.validStrings.isEmpty() ? "any non-duplicate values" : validString.toString(); + + List modifiers = new ArrayList<>(); + if (isEmptyAllowed) + modifiers.add("empty list"); + if (isNullAllowed) + modifiers.add("null"); + + return modifiers.isEmpty() ? base : base + ", " + String.join(", ", modifiers); } } From 7e6aa85c0568b14bb4f0afcf51b923a6a1ae998b Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 13 Nov 2025 12:49:41 +0800 Subject: [PATCH 16/16] update the kafka site message --- .../org/apache/kafka/common/config/ConfigDef.java | 11 +++-------- docs/upgrade.html | 2 +- 2 files changed, 4 insertions(+), 9 deletions(-) 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 3974d733adeac..9ea30790108b3 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 @@ -1082,14 +1082,9 @@ private void validateIndividualValues(String name, List values) { public String toString() { String base = validString.validStrings.isEmpty() ? "any non-duplicate values" : validString.toString(); - - List modifiers = new ArrayList<>(); - if (isEmptyAllowed) - modifiers.add("empty list"); - if (isNullAllowed) - modifiers.add("null"); - - return modifiers.isEmpty() ? base : base + ", " + String.join(", ", modifiers); + String emptyList = isEmptyAllowed ? ", empty list" : ""; + String nullValue = isNullAllowed ? ", null" : ""; + return base + emptyList + nullValue; } } diff --git a/docs/upgrade.html b/docs/upgrade.html index 041928c33884e..a657161acce5d 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -141,7 +141,7 @@
Notable changes in 4 allow a null default value or where a null value has a well-defined semantic meaning.
  • Most LIST-type configurations no longer accept duplicate entries, except in cases where duplicates are explicitly supported. However, if users configure duplicate entries, the internal deduplication - logic will still handle them
  • + logic will still handle them.
  • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined semantic meaning.