From 304c11e339b98a16045a5a14c6334f1ce61d9113 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 19:41:28 +0800 Subject: [PATCH 01/78] add new list validator method --- .../apache/kafka/common/config/ConfigDef.java | 38 ++++++++++++-- .../kafka/common/config/ConfigDefTest.java | 49 +++++++++++++++++++ 2 files changed, 84 insertions(+), 3 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 970d9cebf7231..a42477215033e 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -1006,26 +1006,58 @@ else if (max == null) public static class ValidList implements Validator { final ValidString validString; + final boolean isEmptyAllowed; + final boolean isNullAllowed; - private ValidList(List validStrings) { + private ValidList(List validStrings, boolean isEmptyAllowed, boolean isNullAllowed) { this.validString = new ValidString(validStrings); + this.isEmptyAllowed = isEmptyAllowed; + this.isNullAllowed = isNullAllowed; + } + + public static ValidList anyNonDuplicateValues(boolean isEmptyAllowed, boolean isNullAllowed) { + return new ValidList(List.of(), isEmptyAllowed, isNullAllowed); } public static ValidList in(String... validStrings) { - return new ValidList(Arrays.asList(validStrings)); + return new ValidList(List.of(validStrings), true, false); + } + + public static ValidList in(boolean isEmptyAllowed, String... validStrings) { + if (validStrings.length == 0) { + throw new IllegalArgumentException("Valid strings list cannot be empty for inNonEmpty validator"); + } + return new ValidList(List.of(validStrings), isEmptyAllowed, false); } @Override public void ensureValid(final String name, final Object value) { + if (value == null && isNullAllowed) { + return; + } else if (value == null) { + throw new ConfigException("Configuration '" + name + "' values must not be null."); + } + @SuppressWarnings("unchecked") List values = (List) value; + if (!isEmptyAllowed && values.isEmpty()) { + String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString(); + throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); + } + if (Set.copyOf(values).size() != values.size()) { + throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); + } + if (validString.validStrings.isEmpty()) { + return; + } for (String string : values) { validString.ensureValid(name, string); } } public String toString() { - return validString.toString(); + return validString + (isEmptyAllowed ? " (empty config allowed)" : " (empty not allowed)") + + (isNullAllowed ? " (null config allowed)" : " (null not allowed)"); } } diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index 46791c18471f9..61c2117e0984e 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -758,4 +758,53 @@ public void testListSizeValidatorToString() { assertEquals("List containing maximum of 5 elements", ListSize.atMostOfSize(5).toString()); } + @Test + public void testListValidatorAnyNonDuplicateValues() { + ConfigDef.ValidList allowAnyNonDuplicateValues = ConfigDef.ValidList.anyNonDuplicateValues(true, true); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "b", "c"))); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of())); + assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); + + 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 exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception2.getMessage()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception3.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 exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception5.getMessage()); + } + + @Test + public void testListValidatorIn() { + ConfigDef.ValidList allowEmptyValidator = ConfigDef.ValidList.in(true, "a", "b", "c"); + assertDoesNotThrow(() -> allowEmptyValidator.ensureValid("test.config", List.of("a", "b"))); + assertDoesNotThrow(() -> allowEmptyValidator.ensureValid("test.config", List.of())); + ConfigException exception1 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception1.getMessage()); + ConfigException exception2 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", List.of("d"))); + assertEquals("Invalid value d for configuration test.config: String must be one of: a, b, c", exception2.getMessage()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowEmptyValidator.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception3.getMessage()); + + ConfigDef.ValidList notAllowEmptyValidator = ConfigDef.ValidList.in(false, "a", "b", "c"); + assertDoesNotThrow(() -> notAllowEmptyValidator.ensureValid("test.config", List.of("a", "b"))); + ConfigException exception4 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: [a, b, c]", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception5.getMessage()); + ConfigException exception6 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of("d"))); + assertEquals("Invalid value d for configuration test.config: String must be one of: a, b, c", exception6.getMessage()); + ConfigException exception7 = assertThrows(ConfigException.class, () -> notAllowEmptyValidator.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception7.getMessage()); + } } From 68f281f674c138fc3ff96d83c82b92b6a6101c58 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 19:47:23 +0800 Subject: [PATCH 02/78] sasl.oauthbearer.expected.audience --- .../java/org/apache/kafka/common/config/SaslConfigs.java | 4 +++- .../kafka/common/config/internals/BrokerSecurityConfigs.java | 2 +- .../common/security/oauthbearer/BrokerJwtValidator.java | 5 ++--- 3 files changed, 6 insertions(+), 5 deletions(-) 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 b78d96eaac8fb..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 @@ -19,6 +19,8 @@ import org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString; import org.apache.kafka.common.config.ConfigDef.Range; +import java.util.List; + public class SaslConfigs { private static final String OAUTHBEARER_NOTE = " Currently applies only to OAUTHBEARER."; @@ -407,7 +409,7 @@ public 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, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, ConfigDef.Type.STRING, null, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Type.BOOLEAN, DEFAULT_SASL_OAUTHBEARER_HEADER_URLENCODE, ConfigDef.Importance.LOW, SASL_OAUTHBEARER_HEADER_URLENCODE_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index a3a0baf1e39db..ce5cb6a807856 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 @@ -218,6 +218,6 @@ public class BrokerSecurityConfigs { .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LONG, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslConfigs.SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, INT, SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, LOW, SaslConfigs.SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS_DOC) - .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) + .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE, LIST, List.of(), LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_AUDIENCE_DOC) .define(SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER, STRING, null, LOW, SaslConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java index 23ebca7405003..c69db033052a7 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/BrokerJwtValidator.java @@ -106,8 +106,7 @@ public BrokerJwtValidator() { @Override public void configure(Map configs, String saslMechanism, List jaasConfigEntries) { ConfigurationUtils cu = new ConfigurationUtils(configs, saslMechanism); - List expectedAudiencesList = cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE); - Set expectedAudiences = expectedAudiencesList != null ? Set.copyOf(expectedAudiencesList) : null; + Set expectedAudiences = Set.copyOf(cu.get(SASL_OAUTHBEARER_EXPECTED_AUDIENCE)); Integer clockSkew = cu.validateInteger(SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS, false); String expectedIssuer = cu.validateString(SASL_OAUTHBEARER_EXPECTED_ISSUER, false); String scopeClaimName = cu.validateString(SASL_OAUTHBEARER_SCOPE_CLAIM_NAME); @@ -122,7 +121,7 @@ public void configure(Map configs, String saslMechanism, List Date: Sun, 10 Aug 2025 19:53:33 +0800 Subject: [PATCH 03/78] ssl.cipher.suites, ssl.enabled.protocols --- .../main/java/org/apache/kafka/common/config/SslConfigs.java | 5 +++-- .../kafka/common/config/internals/BrokerSecurityConfigs.java | 2 +- .../kafka/common/security/ssl/DefaultSslEngineFactory.java | 4 ++-- .../org/apache/kafka/connect/runtime/rest/util/SSLUtils.java | 3 +-- 4 files changed, 7 insertions(+), 7 deletions(-) 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 0b88f44f31059..f1bc1ef2e6309 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/SslConfigs.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; +import java.util.List; import java.util.Set; import javax.net.ssl.KeyManagerFactory; @@ -123,8 +124,8 @@ public class SslConfigs { public static void addClientSslSupport(ConfigDef config) { config.define(SslConfigs.SSL_PROTOCOL_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROTOCOL_DOC) .define(SslConfigs.SSL_PROVIDER_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_PROVIDER_DOC) - .define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, null, ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC) - .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_CIPHER_SUITES_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, SslConfigs.SSL_CIPHER_SUITES_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, ConfigDef.Type.LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false),ConfigDef.Importance.MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, ConfigDef.Type.STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, ConfigDef.Importance.MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC) .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, ConfigDef.Type.PASSWORD, null, ConfigDef.Importance.HIGH, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java index ce5cb6a807856..bc55712c9481d 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java +++ b/clients/src/main/java/org/apache/kafka/common/config/internals/BrokerSecurityConfigs.java @@ -154,7 +154,7 @@ public class BrokerSecurityConfigs { .define(BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_CONFIG, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, BrokerSecurityConfigs.SSL_ALLOW_SAN_CHANGES_DOC) .define(SslConfigs.SSL_PROTOCOL_CONFIG, STRING, SslConfigs.DEFAULT_SSL_PROTOCOL, MEDIUM, SslConfigs.SSL_PROTOCOL_DOC) .define(SslConfigs.SSL_PROVIDER_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_PROVIDER_DOC) - .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) + .define(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, LIST, SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS, ConfigDef.ValidList.anyNonDuplicateValues(true, false),MEDIUM, SslConfigs.SSL_ENABLED_PROTOCOLS_DOC) .define(SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, STRING, SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE, MEDIUM, SslConfigs.SSL_KEYSTORE_TYPE_DOC) .define(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, STRING, null, MEDIUM, SslConfigs.SSL_KEYSTORE_LOCATION_DOC) .define(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, PASSWORD, null, MEDIUM, SslConfigs.SSL_KEYSTORE_PASSWORD_DOC) diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java index 0a3d587df9060..a5068dc83abce 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java @@ -137,14 +137,14 @@ public void configure(Map configs) { SecurityUtils.addConfiguredSecurityProviders(this.configs); List cipherSuitesList = (List) configs.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (cipherSuitesList != null && !cipherSuitesList.isEmpty()) { + if (!cipherSuitesList.isEmpty()) { this.cipherSuites = cipherSuitesList.toArray(new String[0]); } else { this.cipherSuites = null; } List enabledProtocolsList = (List) configs.get(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); - if (enabledProtocolsList != null && !enabledProtocolsList.isEmpty()) { + if (!enabledProtocolsList.isEmpty()) { this.enabledProtocols = enabledProtocolsList.toArray(new String[0]); } else { this.enabledProtocols = null; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java index 8f51b6e1b942c..5792e83717104 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java @@ -133,8 +133,7 @@ protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl ssl.setProtocol((String) getOrDefault(sslConfigValues, SslConfigs.SSL_PROTOCOL_CONFIG, SslConfigs.DEFAULT_SSL_PROTOCOL)); List sslCipherSuites = (List) sslConfigValues.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (sslCipherSuites != null) - ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[0])); + ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[0])); ssl.setKeyManagerFactoryAlgorithm((String) getOrDefault(sslConfigValues, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM)); From 6cbf14b50b12e55885d28d1190e14251ff6e0aa2 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 19:56:19 +0800 Subject: [PATCH 04/78] process.roles --- core/src/main/scala/kafka/server/KafkaConfig.scala | 9 +-------- .../org/apache/kafka/server/config/KRaftConfigs.java | 2 +- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 67c6febe1ca60..fe2eac2589a05 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -222,14 +222,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) case role => throw new ConfigException(s"Unknown process role '$role'" + " (only 'broker' and 'controller' are allowed roles)") } - - val distinctRoles: Set[ProcessRole] = roles.toSet - - if (distinctRoles.size != roles.size) { - throw new ConfigException(s"Duplicate role names found in `${KRaftConfigs.PROCESS_ROLES_CONFIG}`: $roles") - } - - distinctRoles + roles.toSet } def isKRaftCombinedMode: Boolean = { 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 9e9580e6d75fd..9dc886fb2ef2d 100644 --- a/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java +++ b/server/src/main/java/org/apache/kafka/server/config/KRaftConfigs.java @@ -70,7 +70,7 @@ public class KRaftConfigs { public static final String CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DOC = "We will log an error message about controller events that take longer than this threshold."; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(PROCESS_ROLES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.in("broker", "controller"), HIGH, PROCESS_ROLES_DOC) + .define(PROCESS_ROLES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.in(false, "broker", "controller"), HIGH, PROCESS_ROLES_DOC) .define(NODE_ID_CONFIG, INT, ConfigDef.NO_DEFAULT_VALUE, atLeast(0), HIGH, NODE_ID_DOC) .define(INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, INT, INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT, null, MEDIUM, INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC) .define(BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, INT, BROKER_HEARTBEAT_INTERVAL_MS_DEFAULT, null, MEDIUM, BROKER_HEARTBEAT_INTERVAL_MS_DOC) From 8e196f11150299ccc6ec550e94ccdfb24a5c7834 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:01:15 +0800 Subject: [PATCH 05/78] StreamsConfig --- .../main/java/org/apache/kafka/streams/StreamsConfig.java | 5 +++++ 1 file changed, 5 insertions(+) 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 48822a996d514..3662b56756588 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -78,6 +78,7 @@ import static org.apache.kafka.common.IsolationLevel.READ_COMMITTED; import static org.apache.kafka.common.config.ConfigDef.ListSize.atMostOfSize; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; import static org.apache.kafka.common.config.ConfigDef.parseType; @@ -901,6 +902,8 @@ public class StreamsConfig extends AbstractConfig { APPLICATION_ID_DOC) .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value Type.LIST, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, @@ -1099,6 +1102,7 @@ public class StreamsConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, @@ -1175,6 +1179,7 @@ public class StreamsConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, From 17bbcc3184b08642b6f1b255f0acf347278c70ae Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:08:20 +0800 Subject: [PATCH 06/78] GroupCoordinatorConfig --- .../java/org/apache/kafka/common/config/SslConfigs.java | 2 +- .../kafka/coordinator/group/GroupCoordinatorConfig.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) 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 f1bc1ef2e6309..dda04651be14c 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 @@ -125,7 +125,7 @@ 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_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/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 f3adcae3fc529..11dd3e30e3f90 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorConfig.java @@ -298,7 +298,7 @@ public class GroupCoordinatorConfig { public static final ConfigDef CONFIG_DEF = new ConfigDef() // Group coordinator configs .define(GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG, LIST, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DEFAULT, - ConfigDef.ValidList.in(Group.GroupType.documentValidValues()), MEDIUM, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) + ConfigDef.ValidList.in(false, Group.GroupType.documentValidValues()), MEDIUM, GROUP_COORDINATOR_REBALANCE_PROTOCOLS_DOC) .define(GROUP_COORDINATOR_NUM_THREADS_CONFIG, INT, GROUP_COORDINATOR_NUM_THREADS_DEFAULT, atLeast(1), HIGH, GROUP_COORDINATOR_NUM_THREADS_DOC) .define(GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG, INT, GROUP_COORDINATOR_APPEND_LINGER_MS_DEFAULT, atLeast(0), MEDIUM, GROUP_COORDINATOR_APPEND_LINGER_MS_DOC) .define(OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, OFFSET_COMMIT_TIMEOUT_MS_DOC) @@ -327,7 +327,7 @@ public class GroupCoordinatorConfig { .define(CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DOC) .define(CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC) .define(CONSUMER_GROUP_MAX_SIZE_CONFIG, INT, CONSUMER_GROUP_MAX_SIZE_DEFAULT, atLeast(1), MEDIUM, CONSUMER_GROUP_MAX_SIZE_DOC) - .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) + .define(CONSUMER_GROUP_ASSIGNORS_CONFIG, LIST, CONSUMER_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, CONSUMER_GROUP_ASSIGNORS_DOC) .define(CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, STRING, CONSUMER_GROUP_MIGRATION_POLICY_DEFAULT, ConfigDef.CaseInsensitiveValidString.in(Utils.enumOptions(ConsumerGroupMigrationPolicy.class)), MEDIUM, CONSUMER_GROUP_MIGRATION_POLICY_DOC) // Interval config used for testing purposes. .defineInternal(CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_CONFIG, INT, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DEFAULT, atLeast(10 * 1000), MEDIUM, CONSUMER_GROUP_REGEX_REFRESH_INTERVAL_MS_DOC) @@ -340,7 +340,7 @@ public class GroupCoordinatorConfig { .define(SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_DOC) .define(SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DEFAULT, atLeast(1), MEDIUM, SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_DOC) .define(SHARE_GROUP_MAX_SIZE_CONFIG, INT, SHARE_GROUP_MAX_SIZE_DEFAULT, between(1, 1000), MEDIUM, SHARE_GROUP_MAX_SIZE_DOC) - .define(SHARE_GROUP_ASSIGNORS_CONFIG, LIST, SHARE_GROUP_ASSIGNORS_DEFAULT, null, MEDIUM, SHARE_GROUP_ASSIGNORS_DOC) + .define(SHARE_GROUP_ASSIGNORS_CONFIG, LIST, SHARE_GROUP_ASSIGNORS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), MEDIUM, SHARE_GROUP_ASSIGNORS_DOC) .defineInternal(SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_CONFIG, INT, SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_DEFAULT, atLeast(1), LOW, SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_DOC) // Streams group configs From 92f9b955b4b02f75b1f44c18d0de9c1be66532c3 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:19:43 +0800 Subject: [PATCH 07/78] fix stream test --- .../kafka/common/config/internals/BrokerSecurityConfigs.java | 2 +- .../kafka/streams/processor/api/MockProcessorContext.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 bc55712c9481d..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.anyNonDuplicateValues(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/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java index 52a2308dafe4c..3ffa85a45034c 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/api/MockProcessorContext.java @@ -214,7 +214,7 @@ public MockProcessorContext() { this( mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "dummy-bootstrap-host:0") )), new TaskId(0, 0), null From a2aef1826cde331a47b9bc6de207952f0c4dab07 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:27:41 +0800 Subject: [PATCH 08/78] MirrorClientConfig, AdminClientConfig, BrokerApiVersionsCommand$AdminClient --- .../apache/kafka/clients/admin/AdminClientConfig.java | 10 +++++++--- .../kafka/connect/mirror/MirrorClientConfig.java | 4 +++- .../apache/kafka/tools/BrokerApiVersionsCommand.java | 2 +- 3 files changed, 11 insertions(+), 5 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 d883ca1450ed9..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 @@ -155,12 +155,14 @@ public class AdminClientConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - "", + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC). define(BOOTSTRAP_CONTROLLERS_CONFIG, Type.LIST, - "", + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, BOOTSTRAP_CONTROLLERS_DOC) .define(CLIENT_ID_CONFIG, Type.STRING, "", Importance.MEDIUM, CLIENT_ID_DOC) @@ -239,6 +241,7 @@ public class AdminClientConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRICS_RECORDING_LEVEL_CONFIG, @@ -284,7 +287,8 @@ public class AdminClientConfig extends AbstractConfig { METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, - List.of(), + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/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 53a4f9f5f051d..c6a20eb7117e7 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClientConfig.java @@ -32,6 +32,7 @@ import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; /** * Configuration required for {@link MirrorClient} to talk to a given target cluster. @@ -133,7 +134,8 @@ private Map clientConfig(String prefix) { static final ConfigDef CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.STRING, - null, + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( 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 13c30721a57f7..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.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 a58c3c3e448d58c6bd12ca5ff8b392ba614193d1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:36:22 +0800 Subject: [PATCH 09/78] MirrorSourceTaskConfig, MirrorCheckpointTaskConfig, MirrorCheckpointConfig --- .../apache/kafka/connect/mirror/MirrorCheckpointConfig.java | 2 ++ .../kafka/connect/mirror/MirrorCheckpointTaskConfig.java | 5 +++-- .../apache/kafka/connect/mirror/MirrorSourceTaskConfig.java | 5 +++-- 3 files changed, 8 insertions(+), 4 deletions(-) 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 3fb2859d2dd46..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,12 +194,14 @@ private static ConfigDef defineCheckpointConfig(ConfigDef baseConfig) { GROUPS, ConfigDef.Type.LIST, GROUPS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, GROUPS_DOC) .define( GROUPS_EXCLUDE, ConfigDef.Type.LIST, GROUPS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, GROUPS_EXCLUDE_DOC) .define( diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java index 0617a4728c263..c482667379d8c 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 @@ -33,7 +33,7 @@ public MirrorCheckpointTaskConfig(Map props) { Set taskConsumerGroups() { List fields = getList(TASK_CONSUMER_GROUPS); - if (fields == null || fields.isEmpty()) { + if (fields.isEmpty()) { return Set.of(); } return new HashSet<>(fields); @@ -54,7 +54,8 @@ String entityLabel() { .define( TASK_CONSUMER_GROUPS, ConfigDef.Type.LIST, - null, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.LOW, TASK_CONSUMER_GROUPS_DOC) .define(TASK_INDEX, diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java index 3d99d12cdd56d..0986d76cf717a 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTaskConfig.java @@ -34,7 +34,7 @@ public MirrorSourceTaskConfig(Map props) { Set taskTopicPartitions() { List fields = getList(TASK_TOPIC_PARTITIONS); - if (fields == null || fields.isEmpty()) { + if (fields.isEmpty()) { return Set.of(); } return fields.stream() @@ -57,7 +57,8 @@ String entityLabel() { .define( TASK_TOPIC_PARTITIONS, ConfigDef.Type.LIST, - null, + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.LOW, TASK_TOPIC_PARTITIONS_DOC) .define(TASK_INDEX, From 3171385c4806497747b1f260c87fc235fc52062d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 20:52:08 +0800 Subject: [PATCH 10/78] RestServerConfig, ServerConfigs --- .../apache/kafka/connect/runtime/rest/RestServerConfig.java | 3 ++- core/src/main/scala/kafka/server/KafkaConfig.scala | 6 +++--- .../java/org/apache/kafka/server/config/ServerConfigs.java | 5 +++-- 3 files changed, 8 insertions(+), 6 deletions(-) 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 96993c37c5ce6..b5255322666fa 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,7 +143,8 @@ public static void addPublicConfig(ConfigDef configDef) { .define( REST_EXTENSION_CLASSES_CONFIG, ConfigDef.Type.LIST, - "", + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC ).define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index fe2eac2589a05..c1655e4a82075 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -268,10 +268,10 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) val earlyStartListeners: Set[ListenerName] = { val listenersSet = listeners.map(l => ListenerName.normalised(l.listener)).toSet val controllerListenersSet = controllerListeners.map(l => ListenerName.normalised(l.listener)).toSet - Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { + Option(getList(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { case None => controllerListenersSet - case Some(str) => - str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str => + case Some(list) => + list.asScala.map(_.trim()).filterNot(_.isEmpty).map { str => val listenerName = new ListenerName(str) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " + 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 4910f00016b13..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 @@ -33,6 +33,7 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN; import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigDef.Type.LIST; import static org.apache.kafka.common.config.ConfigDef.Type.LONG; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; @@ -133,10 +134,10 @@ public class ServerConfigs { .define(REQUEST_TIMEOUT_MS_CONFIG, INT, REQUEST_TIMEOUT_MS_DEFAULT, HIGH, REQUEST_TIMEOUT_MS_DOC) .define(SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, LONG, DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) .define(SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, LONG, SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) /************* Authorizer Configuration ***********/ .define(AUTHORIZER_CLASS_NAME_CONFIG, STRING, AUTHORIZER_CLASS_NAME_DEFAULT, new ConfigDef.NonNullValidator(), LOW, AUTHORIZER_CLASS_NAME_DOC) - .define(EARLY_START_LISTENERS_CONFIG, STRING, null, HIGH, EARLY_START_LISTENERS_DOC) + .define(EARLY_START_LISTENERS_CONFIG, LIST, null, ConfigDef.ValidList.anyNonDuplicateValues(true, true), HIGH, EARLY_START_LISTENERS_DOC) /************ Rack Configuration ******************/ .define(BROKER_RACK_CONFIG, STRING, null, MEDIUM, BROKER_RACK_DOC) /** ********* Controlled shutdown configuration ***********/ From 61ae799329ad26bf08a482919068ff6bbdd02c91 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 21:27:17 +0800 Subject: [PATCH 11/78] listeners --- .../main/scala/kafka/server/KafkaConfig.scala | 6 +++--- .../main/scala/kafka/utils/CoreUtils.scala | 9 +++++---- .../unit/kafka/server/KafkaConfigTest.scala | 6 +++--- .../kafka/network/SocketServerConfigs.java | 20 +++++++++++-------- .../network/SocketServerConfigsTest.java | 16 +++++++-------- .../kafka/tools/MetadataQuorumCommand.java | 5 +++-- 6 files changed, 34 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index c1655e4a82075..ef9738d8bc4d0 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -435,7 +435,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } def listeners: Seq[Endpoint] = - CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap) + CoreUtils.listenerListToEndPoints(getList(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap) def controllerListeners: Seq[Endpoint] = listeners.filter(l => controllerListenerNames.contains(l.listener)) @@ -452,7 +452,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) def effectiveAdvertisedControllerListeners: Seq[Endpoint] = { val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val controllerAdvertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(java.util.List.of(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) .filter(l => controllerListenerNames.contains(l.listener)) } else { Seq.empty @@ -482,7 +482,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) // Use advertised listeners if defined, fallback to listeners otherwise val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(java.util.List.of(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) } else { listeners } diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 31a8aad7c8041..759f8b4938b4f 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs import org.slf4j.event.Level +import java.util.stream.Collectors import scala.jdk.CollectionConverters._ /** @@ -121,16 +122,16 @@ object CoreUtils { def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun) - def listenerListToEndPoints(listeners: String, securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol]): Seq[Endpoint] = { + def listenerListToEndPoints(listeners: java.util.List[String], securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol]): Seq[Endpoint] = { listenerListToEndPoints(listeners, securityProtocolMap, requireDistinctPorts = true) } - private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: String): Unit = { + private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: java.util.List[String]): Unit = { val distinctPorts = endpoints.map(_.port).distinct - require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: $listeners") + require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: ${listeners.stream().collect(Collectors.joining(","))}") } - def listenerListToEndPoints(listeners: String, securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = { + def listenerListToEndPoints(listeners: java.util.List[String], securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = { def validateOneIsIpv4AndOtherIpv6(first: String, second: String): Boolean = (inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) || (inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second)) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 43384a64789c0..f7b6d4afe5607 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -604,7 +604,7 @@ class KafkaConfigTest { assertEquals(expected.securityProtocol(), actual.securityProtocol(), "Security protocol mismatch") } - private def listenerListToEndPoints(listenerList: String, + private def listenerListToEndPoints(listenerList: java.util.List[String], securityProtocolMap: util.Map[ListenerName, SecurityProtocol] = SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO) = CoreUtils.listenerListToEndPoints(listenerList, securityProtocolMap) @@ -618,9 +618,9 @@ class KafkaConfigTest { // configuration with no listeners val conf = KafkaConfig.fromProps(props) - assertEquals(listenerListToEndPoints("PLAINTEXT://:9092"), conf.listeners) + assertEquals(listenerListToEndPoints(util.List.of("PLAINTEXT://:9092")), conf.listeners) assertNull(conf.listeners.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get.host) - assertEquals(conf.effectiveAdvertisedBrokerListeners, listenerListToEndPoints("PLAINTEXT://:9092")) + assertEquals(conf.effectiveAdvertisedBrokerListeners, listenerListToEndPoints(util.List.of("PLAINTEXT://:9092"))) } private def isValidKafkaConfig(props: Properties): Boolean = { 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 e9c3dfff92610..7e59714477839 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -21,7 +21,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.server.util.Csv; import java.util.ArrayList; import java.util.Arrays; @@ -39,6 +38,7 @@ import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Type.INT; +import static org.apache.kafka.common.config.ConfigDef.Type.LIST; import static org.apache.kafka.common.config.ConfigDef.Type.LONG; import static org.apache.kafka.common.config.ConfigDef.Type.STRING; @@ -63,7 +63,7 @@ public class SocketServerConfigs { "is assumed if no explicit mapping is provided and no other security protocol is in use."; public static final String LISTENERS_CONFIG = "listeners"; - public static final String LISTENERS_DEFAULT = "PLAINTEXT://:9092"; + public static final List LISTENERS_DEFAULT = List.of("PLAINTEXT://:9092"); public static final String LISTENERS_DOC = String.format("Listener List - Comma-separated list of URIs we will listen on and the listener names." + " If the listener name is not a security protocol, %s must also be set.%n" + " Listener names and port numbers must be unique unless one listener is an IPv4 address and the other listener is an IPv6 address (for the same port).%n" + @@ -155,7 +155,7 @@ 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, STRING, LISTENERS_DEFAULT, HIGH, LISTENERS_DOC) + .define(LISTENERS_CONFIG, LIST, LISTENERS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, LISTENERS_DOC) .define(ADVERTISED_LISTENERS_CONFIG, STRING, null, 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) @@ -186,7 +186,7 @@ public class SocketServerConfigs { } public static List listenerListToEndPoints( - String input, + List input, Map nameToSecurityProto ) { return listenerListToEndPoints(input, n -> { @@ -199,14 +199,18 @@ public static List listenerListToEndPoints( } public static List listenerListToEndPoints( - String input, + List input, Function nameToSecurityProto ) { List results = new ArrayList<>(); - for (String entry : Csv.parseCsvList(input.trim())) { - Matcher matcher = URI_PARSE_REGEXP.matcher(entry); + for (String entry : input) { + String trimEntry = entry.trim(); + if (trimEntry.isEmpty()) { + continue; + } + Matcher matcher = URI_PARSE_REGEXP.matcher(trimEntry); if (!matcher.matches()) { - throw new KafkaException("Unable to parse " + entry + " to a broker endpoint"); + throw new KafkaException("Unable to parse " + trimEntry + " to a broker endpoint"); } ListenerName listenerName = ListenerName.normalised(matcher.group(1)); String host = matcher.group(2); diff --git a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java index be4184a920226..7b65759a67e36 100644 --- a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java +++ b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java @@ -42,21 +42,21 @@ public void testDefaultNameToSecurityProto() { @Test public void testListenerListToEndPointsWithEmptyString() { assertEquals(List.of(), - SocketServerConfigs.listenerListToEndPoints("", + SocketServerConfigs.listenerListToEndPoints(List.of(), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithBlankString() { assertEquals(List.of(), - SocketServerConfigs.listenerListToEndPoints(" ", + SocketServerConfigs.listenerListToEndPoints(List.of(" "), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithOneEndpoint() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 8080)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://example.com:8080"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @@ -66,35 +66,35 @@ public void testListenerListToEndPointsWithUnderscores() { assertEquals(List.of( new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "example.com", 8080), new Endpoint("SSL", SecurityProtocol.SSL, "local_host", 8081)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://example.com:8080,SSL://local_host:8081", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://example.com:8080", "SSL://local_host:8081"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithWildcard() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, null, 8080)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://:8080", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://:8080"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testListenerListToEndPointsWithIpV6() { assertEquals(List.of(new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "::1", 9092)), - SocketServerConfigs.listenerListToEndPoints("PLAINTEXT://[::1]:9092", + SocketServerConfigs.listenerListToEndPoints(List.of("PLAINTEXT://[::1]:9092"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testAnotherListenerListToEndPointsWithIpV6() { assertEquals(List.of(new Endpoint("SASL_SSL", SecurityProtocol.SASL_SSL, "fe80::b1da:69ca:57f7:63d8%3", 9092)), - SocketServerConfigs.listenerListToEndPoints("SASL_SSL://[fe80::b1da:69ca:57f7:63d8%3]:9092", + SocketServerConfigs.listenerListToEndPoints(List.of("SASL_SSL://[fe80::b1da:69ca:57f7:63d8%3]:9092"), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); } @Test public void testAnotherListenerListToEndPointsWithNonDefaultProtoMap() { assertEquals(List.of(new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "example.com", 9093)), - SocketServerConfigs.listenerListToEndPoints("CONTROLLER://example.com:9093", + SocketServerConfigs.listenerListToEndPoints(List.of("CONTROLLER://example.com:9093"), Map.of(new ListenerName("CONTROLLER"), SecurityProtocol.PLAINTEXT))); } } diff --git a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java index b65655a8c898d..db9b2e6b06769 100644 --- a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java @@ -44,6 +44,7 @@ import net.sourceforge.argparse4j.inf.Subparser; import net.sourceforge.argparse4j.inf.Subparsers; import net.sourceforge.argparse4j.internal.HelpScreenException; +import org.apache.kafka.server.util.Csv; import java.io.File; import java.io.IOException; @@ -396,10 +397,10 @@ static Set getControllerAdvertisedListeners( ) throws Exception { Map listeners = new HashMap<>(); SocketServerConfigs.listenerListToEndPoints( - props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG, "").toString(), + Csv.parseCsvList(props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG, "").toString()), __ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e)); SocketServerConfigs.listenerListToEndPoints( - props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "").toString(), + Csv.parseCsvList(props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "").toString()), __ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e)); if (!props.containsKey(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)) { throw new TerseException(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG + From 7fe1fb34e9c9e529e0542ca577ee07dd05c9437f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:09:37 +0800 Subject: [PATCH 12/78] log.dirs --- core/src/main/scala/kafka/server/KafkaConfig.scala | 5 +++-- .../scala/unit/kafka/server/KafkaConfigTest.scala | 6 +++--- .../kafka/server/config/AbstractKafkaConfig.java | 12 ++++++++++-- .../kafka/storage/internals/log/LogConfig.java | 2 +- 4 files changed, 17 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index ef9738d8bc4d0..fcf9f425df669 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -48,6 +48,7 @@ import org.apache.kafka.server.config.AbstractKafkaConfig.getMap import org.apache.kafka.server.config.{AbstractKafkaConfig, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs +import org.apache.kafka.server.util.Csv import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import scala.jdk.CollectionConverters._ @@ -452,7 +453,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) def effectiveAdvertisedControllerListeners: Seq[Endpoint] = { val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val controllerAdvertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(java.util.List.of(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(Csv.parseCsvList(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) .filter(l => controllerListenerNames.contains(l.listener)) } else { Seq.empty @@ -482,7 +483,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) // Use advertised listeners if defined, fallback to listeners otherwise val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(java.util.List.of(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(Csv.parseCsvList(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) } else { listeners } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index f7b6d4afe5607..46350bdf82ab6 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -245,8 +245,8 @@ class KafkaConfigTest { assertTrue(caught.getMessage.contains("If you have two listeners on the same port then one needs to be IPv4 and the other IPv6")) props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,PLAINTEXT://127.0.0.1:9092") - caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) - assertTrue(caught.getMessage.contains("Each listener must have a different name")) + val exception = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) + assertTrue(exception.getMessage.contains("values must not be duplicated.")) props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9092,SASL_SSL://127.0.0.1:9092") caught = assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)) @@ -824,7 +824,7 @@ class KafkaConfigTest { case SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.NUM_PARTITIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore string + case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore list case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 16d617227273e..231c1cbf86776 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -39,6 +39,7 @@ import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -79,7 +80,13 @@ public AbstractKafkaConfig(ConfigDef definition, Map originals, Map logDirs() { - return Csv.parseCsvList(Optional.ofNullable(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).orElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))); + return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)) + .orElse(Arrays.stream(getString(ServerLogConfigs.LOG_DIR_CONFIG).split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).toList() + ) + .stream() + .map(String::trim) + .toList(); } public int numIoThreads() { @@ -137,7 +144,8 @@ public Map effectiveListenerSecurityProtocolMap( // 2. No SSL or SASL protocols are used in regular listeners (Note: controller listeners // are not included in 'listeners' config when process.roles=broker) if (controllerListenerNames().stream().anyMatch(AbstractKafkaConfig::isSslOrSasl) || - Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).stream() + getList(SocketServerConfigs.LISTENERS_CONFIG).stream() + .map(String::trim) .anyMatch(listenerName -> isSslOrSasl(parseListenerName(listenerName)))) { return mapValue; } else { 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 363fb06c2de28..0f33975e72cdb 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 @@ -145,7 +145,7 @@ 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, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC) - .define(ServerLogConfigs.LOG_DIRS_CONFIG, STRING, null, HIGH, ServerLogConfigs.LOG_DIRS_DOC) + .define(ServerLogConfigs.LOG_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) From a3b392056ac96ce84b4b2f7d200d7267a606f30b Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:21:21 +0800 Subject: [PATCH 13/78] ClientMetricsConfigs, MetricConfigs, ReplaceField, MaskField --- .../apache/kafka/connect/transforms/MaskField.java | 4 ++-- .../kafka/connect/transforms/ReplaceField.java | 12 ++++++++++-- .../scala/kafka/metrics/KafkaMetricsConfig.scala | 2 +- .../kafka/server/metrics/ClientMetricsConfigs.java | 14 ++++++++++++-- .../apache/kafka/server/metrics/MetricConfigs.java | 8 +++++--- 5 files changed, 30 insertions(+), 10 deletions(-) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java index df76a923d81f4..d1b5abf0b00aa 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java @@ -25,7 +25,6 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Values; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.math.BigDecimal; @@ -55,7 +54,8 @@ public abstract class MaskField> implements Transform private static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(true, 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 9584ecfc97862..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,9 +55,17 @@ interface ConfigName { } public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(ConfigName.EXCLUDE, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.MEDIUM, + .define(ConfigName.EXCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.MEDIUM, "Fields to exclude. This takes precedence over the fields to include.") - .define(ConfigName.INCLUDE, ConfigDef.Type.LIST, List.of(), ConfigDef.Importance.MEDIUM, + .define(ConfigName.INCLUDE, + ConfigDef.Type.LIST, + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + ConfigDef.Importance.MEDIUM, "Fields to include. If specified, only these fields will be used.") .define(ConfigName.RENAMES, ConfigDef.Type.LIST, List.of(), ConfigDef.LambdaValidator.with( diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index 83973ede23dbf..c3eb15082175d 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -34,7 +34,7 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * classpath and will be instantiated at run-time. */ val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT)).asScala + String.join("", MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT))).asScala /** * The metrics polling interval (in seconds). 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 2560854eb5558..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,9 +102,19 @@ public class ClientMetricsConfigs extends AbstractConfig { ); private static final ConfigDef CONFIG = new ConfigDef() - .define(METRICS_CONFIG, Type.LIST, METRICS_DEFAULT, Importance.MEDIUM, "Telemetry metric name prefix list") + .define(METRICS_CONFIG, + Type.LIST, + METRICS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.MEDIUM, + "Telemetry metric name prefix list") .define(INTERVAL_MS_CONFIG, Type.INT, INTERVAL_MS_DEFAULT, Importance.MEDIUM, "Metrics push interval in milliseconds") - .define(MATCH_CONFIG, Type.LIST, MATCH_DEFAULT, Importance.MEDIUM, "Client match criteria"); + .define(MATCH_CONFIG, + Type.LIST, + MATCH_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.MEDIUM, + "Client match criteria"); public ClientMetricsConfigs(Properties props) { super(CONFIG, props, false); diff --git a/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java b/server/src/main/java/org/apache/kafka/server/metrics/MetricConfigs.java index 91aca8152879b..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 @@ -21,6 +21,8 @@ import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.Sensor; +import java.util.List; + import static org.apache.kafka.common.config.ConfigDef.Importance.LOW; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Type.INT; @@ -48,7 +50,7 @@ public class MetricConfigs { /** ********* Kafka Yammer Metrics Reporters Configuration ***********/ public static final String KAFKA_METRICS_REPORTER_CLASSES_CONFIG = "kafka.metrics.reporters"; - public static final String KAFKA_METRIC_REPORTER_CLASSES_DEFAULT = ""; + public static final List KAFKA_METRIC_REPORTER_CLASSES_DEFAULT = List.of(); public static final String KAFKA_METRICS_REPORTER_CLASSES_DOC = "A list of classes to use as Yammer metrics custom reporters." + " The reporters should implement kafka.metrics.KafkaMetricsReporter trait. If a client wants" + " to expose JMX operations on a custom reporter, the custom reporter needs to additionally implement an MBean" + @@ -69,11 +71,11 @@ public class MetricConfigs { // Kafka Metrics Configuration .define(METRIC_NUM_SAMPLES_CONFIG, INT, METRIC_NUM_SAMPLES_DEFAULT, atLeast(1), LOW, METRIC_NUM_SAMPLES_DOC) .define(METRIC_SAMPLE_WINDOW_MS_CONFIG, LONG, METRIC_SAMPLE_WINDOW_MS_DEFAULT, atLeast(1), LOW, METRIC_SAMPLE_WINDOW_MS_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, LIST, METRIC_REPORTER_CLASSES_DEFAULT, LOW, METRIC_REPORTER_CLASSES_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, LIST, METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, METRIC_REPORTER_CLASSES_DOC) .define(METRIC_RECORDING_LEVEL_CONFIG, STRING, METRIC_RECORDING_LEVEL_DEFAULT, LOW, METRIC_RECORDING_LEVEL_DOC) // Kafka Yammer Metrics Reporter Configuration - .define(KAFKA_METRICS_REPORTER_CLASSES_CONFIG, LIST, KAFKA_METRIC_REPORTER_CLASSES_DEFAULT, LOW, KAFKA_METRICS_REPORTER_CLASSES_DOC) + .define(KAFKA_METRICS_REPORTER_CLASSES_CONFIG, LIST, KAFKA_METRIC_REPORTER_CLASSES_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), LOW, KAFKA_METRICS_REPORTER_CLASSES_DOC) .define(KAFKA_METRICS_POLLING_INTERVAL_SECONDS_CONFIG, INT, KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DEFAULT, atLeast(1), LOW, KAFKA_METRICS_POLLING_INTERVAL_SECONDS_DOC) // Kafka Client Telemetry Metrics Configuration From 5d3f504298bd2154983cc2b0415e09c25304fc15 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:31:57 +0800 Subject: [PATCH 14/78] MaskField, HeaderFrom, DropHeaders, SinkConnectorConfig, WorkerConfig --- .../kafka/connect/runtime/SinkConnectorConfig.java | 4 ++-- .../org/apache/kafka/connect/runtime/WorkerConfig.java | 10 ++++++++-- .../apache/kafka/connect/transforms/DropHeaders.java | 4 ++-- .../apache/kafka/connect/transforms/HeaderFrom.java | 7 ++++--- .../org/apache/kafka/connect/transforms/MaskField.java | 2 +- 5 files changed, 17 insertions(+), 10 deletions(-) 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 4584255e23132..894591f3e9175 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 @@ -42,7 +42,7 @@ public class SinkConnectorConfig extends ConnectorConfig { public static final String TOPICS_CONFIG = SinkTask.TOPICS_CONFIG; private static final String TOPICS_DOC = "List of topics to consume, separated by commas"; - public static final String TOPICS_DEFAULT = ""; + public static final List TOPICS_DEFAULT = List.of(); private static final String TOPICS_DISPLAY = "Topics"; public static final String TOPICS_REGEX_CONFIG = SinkTask.TOPICS_REGEX_CONFIG; @@ -75,7 +75,7 @@ public class SinkConnectorConfig extends ConnectorConfig { private static ConfigDef configDef(ConfigDef baseConfigs) { return baseConfigs - .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) + .define(TOPICS_CONFIG, ConfigDef.Type.LIST, TOPICS_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_DISPLAY) .define(TOPICS_REGEX_CONFIG, ConfigDef.Type.STRING, TOPICS_REGEX_DEFAULT, new RegexValidator(), ConfigDef.Importance.HIGH, TOPICS_REGEX_DOC, COMMON_GROUP, 4, ConfigDef.Width.LONG, TOPICS_REGEX_DISPLAY) .define(DLQ_TOPIC_NAME_CONFIG, ConfigDef.Type.STRING, DLQ_TOPIC_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_NAME_DOC, ERROR_GROUP, 6, ConfigDef.Width.MEDIUM, DLQ_TOPIC_DISPLAY) .define(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, ConfigDef.Type.SHORT, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DEFAULT, Importance.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DOC, ERROR_GROUP, 7, ConfigDef.Width.MEDIUM, DLQ_TOPIC_REPLICATION_FACTOR_CONFIG_DISPLAY) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index b3aded30682c9..14acbe6d95467 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,6 +200,7 @@ public class WorkerConfig extends AbstractConfig { protected static ConfigDef baseConfigDef() { ConfigDef result = new ConfigDef() .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, @@ -226,6 +227,7 @@ protected static ConfigDef baseConfigDef() { .define(PLUGIN_PATH_CONFIG, Type.LIST, null, + ConfigDef.ValidList.anyNonDuplicateValues(false, true), Importance.LOW, PLUGIN_PATH_DOC) .define(PLUGIN_DISCOVERY_CONFIG, @@ -246,15 +248,19 @@ protected static ConfigDef baseConfigDef() { Importance.LOW, CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, - JmxReporter.class.getName(), Importance.LOW, + JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), + Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(HEADER_CONVERTER_CLASS_CONFIG, Type.CLASS, HEADER_CONVERTER_CLASS_DEFAULT, Importance.LOW, HEADER_CONVERTER_CLASS_DOC) .define(HEADER_CONVERTER_VERSION, Type.STRING, HEADER_CONVERTER_VERSION_DEFAULT, Importance.LOW, HEADER_CONVERTER_VERSION_DOC) - .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, + .define(CONFIG_PROVIDERS_CONFIG, + Type.LIST, Collections.emptyList(), + 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/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 c1d20a48c1d29..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 @@ -23,7 +23,6 @@ import org.apache.kafka.connect.header.ConnectHeaders; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashSet; @@ -41,7 +40,8 @@ public class DropHeaders> implements Transformation> implements Transfor public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELDS_FIELD, ConfigDef.Type.LIST, - NO_DEFAULT_VALUE, new NonEmptyListValidator(), + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, - NO_DEFAULT_VALUE, new NonEmptyListValidator(), + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Header names, in the same order as the field names listed in the fields configuration property.") .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java index d1b5abf0b00aa..7d37d548eb486 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java @@ -55,7 +55,7 @@ public abstract class MaskField> implements Transform public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(true, 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" From 30cf8eed72cc47bc229c4af8a8bbdf3a1d848c62 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:39:27 +0800 Subject: [PATCH 15/78] MirrorMakerConfig, MirrorConnectorConfig, ConsumerConfig, ProducerConfig, MirrorSourceConfig --- .../kafka/clients/consumer/ConsumerConfig.java | 9 +++++---- .../kafka/clients/producer/ProducerConfig.java | 14 ++++++++++---- .../connect/mirror/MirrorConnectorConfig.java | 2 ++ .../kafka/connect/mirror/MirrorMakerConfig.java | 4 ++-- .../kafka/connect/mirror/MirrorSourceConfig.java | 3 +++ 5 files changed, 22 insertions(+), 10 deletions(-) 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 3fcdf20953cba..c71a3d5f6f8b7 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, Collections.emptyList(), - new ConfigDef.NonNullValidator(), + 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), - new ConfigDef.NonNullValidator(), + 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(), - new ConfigDef.NonNullValidator(), + 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, Collections.emptyList(), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(MAX_POLL_RECORDS_CONFIG, @@ -702,6 +702,7 @@ public class ConsumerConfig extends AbstractConfig { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index ff7079bcdab37..ca2583b72df31 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 @@ -373,7 +373,12 @@ public class ProducerConfig extends AbstractConfig { private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); static { - CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, Collections.emptyList(), new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, + Collections.emptyList(), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, ClientDnsLookup.USE_ALL_DNS_IPS.toString(), @@ -462,7 +467,7 @@ public class ProducerConfig extends AbstractConfig { .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, JmxReporter.class.getName(), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define(MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, @@ -502,7 +507,7 @@ public class ProducerConfig extends AbstractConfig { .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, Collections.emptyList(), - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, @@ -555,7 +560,8 @@ public class ProducerConfig extends AbstractConfig { CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, - List.of(), + List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } diff --git a/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 849f40c1fce12..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 @@ -311,6 +311,7 @@ String entityLabel() { CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, ConfigDef.Type.LIST, JmxReporter.class.getName(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) .define( @@ -323,6 +324,7 @@ String entityLabel() { .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC) .withClientSslSupport() diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 01e70e0af9713..a848d9c4bd5c5 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) { protected static ConfigDef config() { ConfigDef result = new ConfigDef() - .define(CLUSTERS_CONFIG, Type.LIST, Importance.HIGH, CLUSTERS_DOC) + .define(CLUSTERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, CLUSTERS_DOC) .define(ENABLE_INTERNAL_REST_CONFIG, Type.BOOLEAN, false, Importance.HIGH, ENABLE_INTERNAL_REST_DOC) - .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), Importance.LOW, CONFIG_PROVIDERS_DOC) + .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, CONFIG_PROVIDERS_DOC) // security support .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, Type.STRING, diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConfig.java index dc0da5382338c..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 @@ -209,18 +209,21 @@ private static ConfigDef defineSourceConfig(ConfigDef baseConfig) { TOPICS, ConfigDef.Type.LIST, TOPICS_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_DOC) .define( TOPICS_EXCLUDE, ConfigDef.Type.LIST, TOPICS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, TOPICS_EXCLUDE_DOC) .define( CONFIG_PROPERTIES_EXCLUDE, ConfigDef.Type.LIST, CONFIG_PROPERTIES_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), ConfigDef.Importance.HIGH, CONFIG_PROPERTIES_EXCLUDE_DOC) .define( From f603b757a03109564b67677b5f05d47a299e5ecd Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:47:35 +0800 Subject: [PATCH 16/78] remove NonEmptyListValidator.java --- .../kafka/connect/transforms/ValueToKey.java | 3 +- .../util/NonEmptyListValidator.java | 38 ---------------- .../util/NonEmptyListValidatorTest.java | 44 ------------------- 3 files changed, 1 insertion(+), 84 deletions(-) delete mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java delete mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java 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 24cdec2249ab1..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 @@ -28,7 +28,6 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; @@ -46,7 +45,7 @@ public class ValueToKey> implements Transformation public static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Field names on the record value to extract as the record key.") .define(REPLACE_NULL_WITH_DEFAULT_CONFIG, ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.MEDIUM, "Whether to replace fields that have a default value and that are null to the default value. When set to true, the default value is used, otherwise null is used."); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java deleted file mode 100644 index dacb07bb8b773..0000000000000 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.connect.transforms.util; - -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigException; - -import java.util.List; - -public class NonEmptyListValidator implements ConfigDef.Validator { - - @Override - public void ensureValid(String name, Object value) { - if (value == null || ((List) value).isEmpty()) { - throw new ConfigException(name, value, "Empty list"); - } - } - - @Override - public String toString() { - return "non-empty list"; - } - -} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java deleted file mode 100644 index 3a9ef48f8dddb..0000000000000 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.connect.transforms.util; - -import org.apache.kafka.common.config.ConfigException; - -import org.junit.jupiter.api.Test; - -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertThrows; - -public class NonEmptyListValidatorTest { - - @Test - public void testNullList() { - assertThrows(ConfigException.class, () -> new NonEmptyListValidator().ensureValid("foo", null)); - } - - @Test - public void testEmptyList() { - assertThrows(ConfigException.class, - () -> new NonEmptyListValidator().ensureValid("foo", List.of())); - } - - @Test - public void testValidList() { - new NonEmptyListValidator().ensureValid("foo", List.of("foo")); - } -} From 322a991241c97b0263345b026b6c0db8b5156775 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 22:50:02 +0800 Subject: [PATCH 17/78] spotlessApply --- .../main/java/org/apache/kafka/tools/MetadataQuorumCommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java index db9b2e6b06769..5c406bdb05b84 100644 --- a/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java +++ b/tools/src/main/java/org/apache/kafka/tools/MetadataQuorumCommand.java @@ -33,6 +33,7 @@ import org.apache.kafka.server.config.KRaftConfigs; import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.util.CommandLineUtils; +import org.apache.kafka.server.util.Csv; import net.sourceforge.argparse4j.ArgumentParsers; import net.sourceforge.argparse4j.impl.Arguments; @@ -44,7 +45,6 @@ import net.sourceforge.argparse4j.inf.Subparser; import net.sourceforge.argparse4j.inf.Subparsers; import net.sourceforge.argparse4j.internal.HelpScreenException; -import org.apache.kafka.server.util.Csv; import java.io.File; import java.io.IOException; From adc5c958ba0502bdb23066cd9ca0f03f8e19e25e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 23:01:46 +0800 Subject: [PATCH 18/78] Revert "remove NonEmptyListValidator.java" This reverts commit f603b757a03109564b67677b5f05d47a299e5ecd. --- .../kafka/connect/transforms/ValueToKey.java | 3 +- .../util/NonEmptyListValidator.java | 38 ++++++++++++++++ .../util/NonEmptyListValidatorTest.java | 44 +++++++++++++++++++ 3 files changed, 84 insertions(+), 1 deletion(-) create mode 100644 connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java create mode 100644 connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java 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..24cdec2249ab1 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -28,6 +28,7 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; @@ -45,7 +46,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, new NonEmptyListValidator(), 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/NonEmptyListValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java new file mode 100644 index 0000000000000..dacb07bb8b773 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.List; + +public class NonEmptyListValidator implements ConfigDef.Validator { + + @Override + public void ensureValid(String name, Object value) { + if (value == null || ((List) value).isEmpty()) { + throw new ConfigException(name, value, "Empty list"); + } + } + + @Override + public String toString() { + return "non-empty list"; + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java new file mode 100644 index 0000000000000..3a9ef48f8dddb --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidatorTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.common.config.ConfigException; + +import org.junit.jupiter.api.Test; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class NonEmptyListValidatorTest { + + @Test + public void testNullList() { + assertThrows(ConfigException.class, () -> new NonEmptyListValidator().ensureValid("foo", null)); + } + + @Test + public void testEmptyList() { + assertThrows(ConfigException.class, + () -> new NonEmptyListValidator().ensureValid("foo", List.of())); + } + + @Test + public void testValidList() { + new NonEmptyListValidator().ensureValid("foo", List.of("foo")); + } +} From c98d261d94c6b2f2d310c4161565020d4cd2d863 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 10 Aug 2025 23:03:03 +0800 Subject: [PATCH 19/78] update HeaderFrom config --- .../java/org/apache/kafka/connect/transforms/HeaderFrom.java | 3 ++- .../java/org/apache/kafka/connect/transforms/ValueToKey.java | 3 +-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java index 4a2a73a77bb2d..ae5dd1c6a067e 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -30,6 +30,7 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.header.Header; import org.apache.kafka.connect.header.Headers; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.Requirements; import org.apache.kafka.connect.transforms.util.SchemaUtil; import org.apache.kafka.connect.transforms.util.SimpleConfig; @@ -61,7 +62,7 @@ public abstract class HeaderFrom> implements Transfor public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELDS_FIELD, ConfigDef.Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, 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 24cdec2249ab1..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 @@ -28,7 +28,6 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; @@ -46,7 +45,7 @@ public class ValueToKey> implements Transformation public static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Field names on the record value to extract as the record key.") .define(REPLACE_NULL_WITH_DEFAULT_CONFIG, ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.MEDIUM, "Whether to replace fields that have a default value and that are null to the default value. When set to true, the default value is used, otherwise null is used."); From 6b3bb4dcd4a5a3d90b1409c448a2f4b82c8dcee9 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 00:14:02 +0800 Subject: [PATCH 20/78] fix consumerConfig BOOTSTRAP_SERVERS_CONFIG --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 c71a3d5f6f8b7..ab5bc684b4187 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 @@ -415,8 +415,8 @@ public class ConsumerConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - Collections.emptyList(), - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + ConfigDef.NO_DEFAULT_VALUE, + null, Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, From 053957b166f52ae7cd77df5b9c5bfa78dc1eeb1f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 09:19:30 +0800 Subject: [PATCH 21/78] fix ConsumerConfig --- .../kafka/clients/consumer/ConsumerConfig.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) 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 ab5bc684b4187..4f5b33c797324 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 @@ -63,9 +63,9 @@ public class ConsumerConfig extends AbstractConfig { // a list contains all the assignor names that only assign subscribed topics to consumer. Should be updated when new assignor added. // This is to help optimize ConsumerCoordinator#performAssignment method public static final List ASSIGN_FROM_SUBSCRIBED_ASSIGNORS = List.of( - RANGE_ASSIGNOR_NAME, - ROUNDROBIN_ASSIGNOR_NAME, - STICKY_ASSIGNOR_NAME, + RANGE_ASSIGNOR_NAME, + ROUNDROBIN_ASSIGNOR_NAME, + STICKY_ASSIGNOR_NAME, COOPERATIVE_STICKY_ASSIGNOR_NAME ); @@ -406,17 +406,17 @@ public class ConsumerConfig extends AbstractConfig { * A list of configuration keys not supported for CONSUMER protocol. */ private static final List CONSUMER_PROTOCOL_UNSUPPORTED_CONFIGS = List.of( - PARTITION_ASSIGNMENT_STRATEGY_CONFIG, - HEARTBEAT_INTERVAL_MS_CONFIG, + PARTITION_ASSIGNMENT_STRATEGY_CONFIG, + HEARTBEAT_INTERVAL_MS_CONFIG, SESSION_TIMEOUT_MS_CONFIG, SHARE_ACKNOWLEDGEMENT_MODE_CONFIG ); - + static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - ConfigDef.NO_DEFAULT_VALUE, - null, + Collections.emptyList(), + new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, From 4952999c0f9de0937c6ff667a07080f641d20bdb Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 11:32:29 +0800 Subject: [PATCH 22/78] update bootstrap.server config --- .../kafka/clients/producer/ProducerConfig.java | 12 ++++++------ .../kafka/connect/mirror/MirrorClientConfig.java | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) 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 ca2583b72df31..a9aaf655ead76 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 @@ -373,11 +373,11 @@ public class ProducerConfig extends AbstractConfig { private static final AtomicInteger PRODUCER_CLIENT_ID_SEQUENCE = new AtomicInteger(1); static { - CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, - Type.LIST, + CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, + Type.LIST, Collections.emptyList(), - ConfigDef.ValidList.anyNonDuplicateValues(false, false), - Importance.HIGH, + new ConfigDef.NonNullValidator(), + Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, Type.STRING, @@ -558,11 +558,11 @@ public class ProducerConfig extends AbstractConfig { atLeast(0), Importance.LOW, CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC) - .define(CONFIG_PROVIDERS_CONFIG, + .define(CONFIG_PROVIDERS_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, + ConfigDef.Importance.LOW, CONFIG_PROVIDERS_DOC); } 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 c6a20eb7117e7..5b0e73b08bd35 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 @@ -29,10 +29,10 @@ import org.apache.kafka.common.utils.Utils; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; -import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; /** * Configuration required for {@link MirrorClient} to talk to a given target cluster. @@ -106,7 +106,7 @@ public Map consumerConfig() { public Map producerConfig() { return clientConfig(PRODUCER_CLIENT_PREFIX); } - + private Map clientConfig(String prefix) { Map props = new HashMap<>(valuesWithPrefixOverride(prefix)); props.keySet().retainAll(CLIENT_CONFIG_DEF.names()); @@ -130,14 +130,14 @@ private Map clientConfig(String prefix) { CommonClientConfigs.SECURITY_PROTOCOL_DOC) .withClientSslSupport() .withClientSaslSupport(); - + static final ConfigDef CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.STRING, - NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + List.of(), + new ConfigDef.NonNullValidator(), Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( REPLICATION_POLICY_CLASS, ConfigDef.Type.CLASS, From 8f067b3881c364b6dc3478225e71b8282cd85174 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 11:57:14 +0800 Subject: [PATCH 23/78] fix some build fail --- .../apache/kafka/connect/runtime/WorkerConfig.java | 13 +++++++------ .../apache/kafka/connect/transforms/HeaderFrom.java | 4 ++-- .../apache/kafka/connect/transforms/ValueToKey.java | 3 ++- .../org/apache/kafka/streams/StreamsConfig.java | 4 ++-- 4 files changed, 13 insertions(+), 11 deletions(-) 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 14acbe6d95467..81db9d37e5714 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 @@ -32,6 +32,7 @@ import org.apache.kafka.connect.runtime.rest.RestServerConfig; import org.apache.kafka.connect.storage.SimpleHeaderConverter; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,13 +60,13 @@ public class WorkerConfig extends AbstractConfig { private static final Logger log = LoggerFactory.getLogger(WorkerConfig.class); public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; - public static final String BOOTSTRAP_SERVERS_DOC = + public static final String BOOTSTRAP_SERVERS_DOC = "A list of host/port pairs used to establish the initial connection to the Kafka cluster. " + "Clients use this list to bootstrap and discover the full set of Kafka brokers. " + "While the order of servers in the list does not matter, we recommend including more than one server to ensure resilience if any servers are down. " + "This list does not need to contain the entire set of brokers, as Kafka clients automatically manage and update connections to the cluster efficiently. " + "This list must be in the form host1:port1,host2:port2,...."; - public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; + public static final List BOOTSTRAP_SERVERS_DEFAULT = List.of("localhost:9092"); public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG; public static final String CLIENT_DNS_LOOKUP_DOC = CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC; @@ -137,7 +138,7 @@ public class WorkerConfig extends AbstractConfig { + "plugins and their dependencies\n" + "Note: symlinks will be followed to discover dependencies or plugins.\n" + "Examples: plugin.path=/usr/local/share/java,/usr/local/share/kafka/plugins," - + "/opt/connectors\n" + + "/opt/connectors\n" + "Do not use config provider variables in this property, since the raw path is used " + "by the worker's scanner before config providers are initialized and used to " + "replace variables."; @@ -257,7 +258,7 @@ protected static ConfigDef baseConfigDef() { Importance.LOW, HEADER_CONVERTER_CLASS_DOC) .define(HEADER_CONVERTER_VERSION, Type.STRING, HEADER_CONVERTER_VERSION_DEFAULT, Importance.LOW, HEADER_CONVERTER_VERSION_DOC) - .define(CONFIG_PROVIDERS_CONFIG, + .define(CONFIG_PROVIDERS_CONFIG, Type.LIST, Collections.emptyList(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), @@ -344,8 +345,8 @@ private void logPluginPathConfigProviderWarning(Map rawOriginals if (!Objects.equals(rawPluginPath, transformedPluginPath)) { log.warn( "Variables cannot be used in the 'plugin.path' property, since the property is " - + "used by plugin scanning before the config providers that replace the " - + "variables are initialized. The raw value '{}' was used for plugin scanning, as " + + "used by plugin scanning before the config providers that replace the " + + "variables are initialized. The raw value '{}' was used for plugin scanning, as " + "opposed to the transformed value '{}', and this may cause unexpected results.", rawPluginPath, transformedPluginPath diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java index ae5dd1c6a067e..a4b40a38bcf43 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -66,8 +66,8 @@ public abstract class HeaderFrom> implements Transfor ConfigDef.Importance.HIGH, "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, - NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + NO_DEFAULT_VALUE, + new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Header names, in the same order as the field names listed in the fields configuration property.") .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, 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..24cdec2249ab1 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -28,6 +28,7 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; @@ -45,7 +46,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, new NonEmptyListValidator(), 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/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 3662b56756588..f8823a436863d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -903,7 +903,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), + new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, @@ -1103,7 +1103,7 @@ public class StreamsConfig extends AbstractConfig { Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), - Importance.LOW, + Importance.LOW, CONFIG_PROVIDERS_DOC) .define(ENABLE_METRICS_PUSH_CONFIG, Type.BOOLEAN, From a986cb933056ea0521fa5856dc77be7108b333a5 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 11:58:14 +0800 Subject: [PATCH 24/78] spotlessApply --- .../main/java/org/apache/kafka/connect/runtime/WorkerConfig.java | 1 - 1 file changed, 1 deletion(-) 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 81db9d37e5714..d48ac2583da9c 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 @@ -32,7 +32,6 @@ import org.apache.kafka.connect.runtime.rest.RestServerConfig; import org.apache.kafka.connect.storage.SimpleHeaderConverter; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 9402ef1406111037f608d15838c46108b780cfe6 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 12:12:07 +0800 Subject: [PATCH 25/78] fix MirrorClientConfig --- .../org/apache/kafka/connect/mirror/MirrorClientConfig.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 5b0e73b08bd35..e84d115a9233e 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 @@ -29,10 +29,10 @@ import org.apache.kafka.common.utils.Utils; import java.util.HashMap; -import java.util.List; import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.CaseInsensitiveValidString.in; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; /** * Configuration required for {@link MirrorClient} to talk to a given target cluster. @@ -134,7 +134,7 @@ private Map clientConfig(String prefix) { static final ConfigDef CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.STRING, - List.of(), + NO_DEFAULT_VALUE, new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) From bd6f2fad16feca2e8be9378cbdb88fc7f5919247 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 20:43:08 +0800 Subject: [PATCH 26/78] fix LogConfigTest and KRaftClusterTest --- .../test/scala/integration/kafka/server/KRaftClusterTest.scala | 2 +- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 2 +- .../java/org/apache/kafka/storage/internals/log/LogConfig.java | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index 55911c1e796a9..dfc999d7c64a7 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -142,7 +142,7 @@ class KRaftClusterTest { cluster.format() cluster.startup() cluster.brokers().forEach((_, broker) => { - assertEquals("external://localhost:0", broker.config.get(SocketServerConfigs.LISTENERS_CONFIG)) + assertEquals(util.List.of("external://localhost:0"), broker.config.get(SocketServerConfigs.LISTENERS_CONFIG)) assertEquals("external", broker.config.get(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG)) assertEquals("external:PLAINTEXT,CONTROLLER:PLAINTEXT", broker.config.get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)) }) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 1b5c61fc777d4..cf358802d978f 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -295,7 +295,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,delete,delete") + logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete") validateCleanupPolicy() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "") assertThrows(classOf[ConfigException], () => validateCleanupPolicy()) 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 0f33975e72cdb..9a83ae1d1b64f 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,8 +561,7 @@ 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 (!Set.of(TopicConfig.CLEANUP_POLICY_DELETE).equals(policySet)) { + if (cleanupPolicy.size() != 1 || !TopicConfig.CLEANUP_POLICY_DELETE.equals(cleanupPolicy.get(0))) { throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete"); } } From 780e96416b141028d17f76938d5df322e150005c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 21:39:48 +0800 Subject: [PATCH 27/78] fix MirrorClientConfig and test --- .../apache/kafka/connect/mirror/MirrorClientConfig.java | 3 ++- .../kafka/connect/mirror/MirrorMakerConfigTest.java | 9 +++++++-- 2 files changed, 9 insertions(+), 3 deletions(-) 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 e84d115a9233e..a7d3708bffc67 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 @@ -118,7 +118,8 @@ private Map clientConfig(String prefix) { static final ConfigDef CLIENT_CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - null, + NO_DEFAULT_VALUE, + new ConfigDef.NonNullValidator(), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) // security support diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java index 7dfdf460c44c6..d728b29f0ba52 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java @@ -316,7 +316,10 @@ public void testInvalidSecurityProtocol() { @Test public void testClientInvalidSecurityProtocol() { ConfigException ce = assertThrows(ConfigException.class, - () -> new MirrorClientConfig(makeProps(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"))); + () -> new MirrorClientConfig(makeProps( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc", + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + ))); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -324,7 +327,9 @@ public void testClientInvalidSecurityProtocol() { public void testCaseInsensitiveSecurityProtocol() { final String saslSslLowerCase = SecurityProtocol.SASL_SSL.name.toLowerCase(Locale.ROOT); final MirrorClientConfig config = new MirrorClientConfig(makeProps( - CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase)); + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase, + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + )); assertEquals(saslSslLowerCase, config.originalsStrings().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } From 27b3091f64f70418d9243fb79db9fb44ea8b0dda Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 21:47:43 +0800 Subject: [PATCH 28/78] fix CommonNameLoggingSslEngineFactoryTest add null check back --- .../kafka/common/security/ssl/DefaultSslEngineFactory.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java index a5068dc83abce..0a3d587df9060 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java @@ -137,14 +137,14 @@ public void configure(Map configs) { SecurityUtils.addConfiguredSecurityProviders(this.configs); List cipherSuitesList = (List) configs.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (!cipherSuitesList.isEmpty()) { + if (cipherSuitesList != null && !cipherSuitesList.isEmpty()) { this.cipherSuites = cipherSuitesList.toArray(new String[0]); } else { this.cipherSuites = null; } List enabledProtocolsList = (List) configs.get(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); - if (!enabledProtocolsList.isEmpty()) { + if (enabledProtocolsList != null && !enabledProtocolsList.isEmpty()) { this.enabledProtocols = enabledProtocolsList.toArray(new String[0]); } else { this.enabledProtocols = null; From 15718a70ff366429cf47098c32ff95a5136fadaa Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 11 Aug 2025 22:29:29 +0800 Subject: [PATCH 29/78] fix KafkaConsumerTest --- .../clients/consumer/KafkaConsumerTest.java | 26 ------------------- 1 file changed, 26 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index acc2616b8c4d5..4069b9d25c1c1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -713,32 +713,6 @@ public void testInterceptorConstructorClose(GroupProtocol groupProtocol) { } } - @ParameterizedTest - @EnumSource(GroupProtocol.class) - public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) { - final int targetInterceptor = 3; - - try { - Properties props = new Properties(); - props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); - props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName() + ", " - + MockConsumerInterceptor.class.getName() + ", " - + MockConsumerInterceptor.class.getName()); - - MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); - - assertThrows(KafkaException.class, () -> newConsumer( - props, new StringDeserializer(), new StringDeserializer())); - - assertEquals(3, MockConsumerInterceptor.CONFIG_COUNT.get()); - assertEquals(3, MockConsumerInterceptor.CLOSE_COUNT.get()); - - } finally { - MockConsumerInterceptor.resetCounters(); - } - } - @ParameterizedTest @EnumSource(GroupProtocol.class) public void testPause(GroupProtocol groupProtocol) { From b395ed286d510f190bc4eab0cf2a5093ccb265de Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 09:23:39 +0800 Subject: [PATCH 30/78] fix KafkaProducerTest, AdminBootstrapAddressesTest --- .../AdminBootstrapAddressesTest.java | 18 ++--- .../clients/producer/KafkaProducerTest.java | 69 +++++++------------ 2 files changed, 30 insertions(+), 57 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java index a61a7bdfda5f1..0581d672fb8a0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/internals/AdminBootstrapAddressesTest.java @@ -32,17 +32,13 @@ import static org.junit.jupiter.api.Assertions.assertThrows; public class AdminBootstrapAddressesTest { - @ParameterizedTest - @ValueSource(booleans = {false, true}) - public void testNoBootstrapSet(boolean nullValue) { - Map map = new HashMap<>(); - if (nullValue) { - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, null); - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, null); - } else { - map.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, ""); - map.put(AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, ""); - } + + @Test + public void testNoBootstrapSet() { + Map map = Map.of( + AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, "", + AdminClientConfig.BOOTSTRAP_CONTROLLERS_CONFIG, "" + ); AdminClientConfig config = new AdminClientConfig(map); assertEquals("You must set either bootstrap.servers or bootstrap.controllers", assertThrows(ConfigException.class, () -> AdminBootstrapAddresses.fromConfig(config)). diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index a52d3c6f0b202..a571f06599e16 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -637,30 +637,7 @@ props, new StringSerializer(), new StringSerializer())) { MockProducerInterceptor.resetCounters(); } } - @Test - public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances() { - final int targetInterceptor = 3; - try { - Properties props = new Properties(); - props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, org.apache.kafka.test.MockProducerInterceptor.class.getName() + ", " - + org.apache.kafka.test.MockProducerInterceptor.class.getName() + ", " - + org.apache.kafka.test.MockProducerInterceptor.class.getName()); - props.setProperty(MockProducerInterceptor.APPEND_STRING_PROP, "something"); - - MockProducerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); - - assertThrows(KafkaException.class, () -> - new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()) - ); - - assertEquals(3, MockProducerInterceptor.CONFIG_COUNT.get()); - assertEquals(3, MockProducerInterceptor.CLOSE_COUNT.get()); - } finally { - MockProducerInterceptor.resetCounters(); - } - } @Test public void testPartitionerClose() { try { @@ -1412,12 +1389,12 @@ public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean kee // Capture flags from the InitProducerIdRequest boolean[] requestFlags = new boolean[2]; // [keepPreparedTxn, enable2Pc] - + client.prepareResponse( request -> request instanceof FindCoordinatorRequest && ((FindCoordinatorRequest) request).data().keyType() == FindCoordinatorRequest.CoordinatorType.TRANSACTION.id(), FindCoordinatorResponse.prepareResponse(Errors.NONE, "test-txn-id", NODE)); - + client.prepareResponse( request -> { if (request instanceof InitProducerIdRequest) { @@ -1429,15 +1406,15 @@ public void testInitTransactionsWithKeepPreparedTxnAndTwoPhaseCommit(boolean kee return false; }, initProducerIdResponse(1L, (short) 5, Errors.NONE)); - + try (Producer producer = kafkaProducer(configs, new StringSerializer(), new StringSerializer(), metadata, client, null, time)) { producer.initTransactions(keepPreparedTxn); - + // Verify request flags match expected values - assertEquals(keepPreparedTxn, requestFlags[0], + assertEquals(keepPreparedTxn, requestFlags[0], "keepPreparedTxn flag should match input parameter"); - assertEquals(enable2PC, requestFlags[1], + assertEquals(enable2PC, requestFlags[1], "enable2Pc flag should match producer configuration"); } } @@ -1603,7 +1580,7 @@ public void testPrepareTransactionFailsWhen2PCDisabled() { ); } } - + @Test public void testCompleteTransactionWithMatchingState() throws Exception { StringSerializer serializer = new StringSerializer(); @@ -1611,37 +1588,37 @@ public void testCompleteTransactionWithMatchingState() throws Exception { when(ctx.transactionManager.isPrepared()).thenReturn(true); when(ctx.sender.isRunning()).thenReturn(true); - + // Create prepared states with matching values long producerId = 12345L; short epoch = 5; PreparedTxnState inputState = new PreparedTxnState(producerId, epoch); ProducerIdAndEpoch currentProducerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); - + // Set up the transaction manager to return the prepared state when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentProducerIdAndEpoch); - + // Should trigger commit when states match TransactionalRequestResult commitResult = mock(TransactionalRequestResult.class); when(ctx.transactionManager.beginCommit()).thenReturn(commitResult); - + try (KafkaProducer producer = ctx.newKafkaProducer()) { // Call completeTransaction with the matching state producer.completeTransaction(inputState); - + // Verify methods called in order verify(ctx.transactionManager).isPrepared(); verify(ctx.transactionManager).preparedTransactionState(); verify(ctx.transactionManager).beginCommit(); - + // Verify abort was never called verify(ctx.transactionManager, never()).beginAbort(); - + // Verify sender was woken up verify(ctx.sender).wakeup(); } } - + @Test public void testCompleteTransactionWithNonMatchingState() throws Exception { StringSerializer serializer = new StringSerializer(); @@ -1649,37 +1626,37 @@ public void testCompleteTransactionWithNonMatchingState() throws Exception { when(ctx.transactionManager.isPrepared()).thenReturn(true); when(ctx.sender.isRunning()).thenReturn(true); - + // Create txn prepared states with different values long producerId = 12345L; short epoch = 5; PreparedTxnState inputState = new PreparedTxnState(producerId + 1, epoch); ProducerIdAndEpoch currentProducerIdAndEpoch = new ProducerIdAndEpoch(producerId, epoch); - + // Set up the transaction manager to return the prepared state when(ctx.transactionManager.preparedTransactionState()).thenReturn(currentProducerIdAndEpoch); - + // Should trigger abort when states don't match TransactionalRequestResult abortResult = mock(TransactionalRequestResult.class); when(ctx.transactionManager.beginAbort()).thenReturn(abortResult); - + try (KafkaProducer producer = ctx.newKafkaProducer()) { // Call completeTransaction with the non-matching state producer.completeTransaction(inputState); - + // Verify methods called in order verify(ctx.transactionManager).isPrepared(); verify(ctx.transactionManager).preparedTransactionState(); verify(ctx.transactionManager).beginAbort(); - + // Verify commit was never called verify(ctx.transactionManager, never()).beginCommit(); - + // Verify sender was woken up verify(ctx.sender).wakeup(); } } - + @Test public void testClusterAuthorizationFailure() throws Exception { int maxBlockMs = 500; From 583955ac624c4a7e4acaf57b97d09bced4a7eb1e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 13:07:11 +0800 Subject: [PATCH 31/78] update BOOTSTRAP_SERVERS_CONFIG related config --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 4 ++-- .../org/apache/kafka/clients/producer/ProducerConfig.java | 5 +++-- .../org/apache/kafka/connect/mirror/MirrorClientConfig.java | 4 ++-- .../main/java/org/apache/kafka/streams/StreamsConfig.java | 2 +- 4 files changed, 8 insertions(+), 7 deletions(-) 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 4f5b33c797324..d72eaa495e793 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 @@ -415,8 +415,8 @@ public class ConsumerConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - Collections.emptyList(), - new ConfigDef.NonNullValidator(), + ConfigDef.NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, 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 a9aaf655ead76..09240d31ccf00 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 @@ -43,6 +43,7 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.kafka.common.config.ConfigDef.NO_DEFAULT_VALUE; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; @@ -375,8 +376,8 @@ public class ProducerConfig extends AbstractConfig { static { CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, - Collections.emptyList(), - new ConfigDef.NonNullValidator(), + NO_DEFAULT_VALUE, + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, 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 a7d3708bffc67..acfbe843b95ed 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, - new ConfigDef.NonNullValidator(), + 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.STRING, NO_DEFAULT_VALUE, - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define( 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 f8823a436863d..1bd533fbd1433 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -903,7 +903,7 @@ public class StreamsConfig extends AbstractConfig { .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value Type.LIST, NO_DEFAULT_VALUE, - new ConfigDef.NonNullValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) .define(NUM_STANDBY_REPLICAS_CONFIG, From 4c7ac76007dbc5d6505ba0ff1325e560ec7079b7 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 16:31:06 +0800 Subject: [PATCH 32/78] update MockProcessorContext related test --- .../apache/kafka/streams/processor/MockProcessorContext.java | 2 +- .../kafka/streams/test/MockProcessorContextAPITest.java | 4 ++-- .../streams/test/MockProcessorContextStateStoreTest.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index fc7d27a3bb792..0ffea9b49169f 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -194,7 +194,7 @@ public MockProcessorContext() { new Properties() { { put(StreamsConfig.APPLICATION_ID_CONFIG, ""); - put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"); } }, new TaskId(0, 0), diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java index 929b5d6533d4c..f2b7f8ee68c65 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextAPITest.java @@ -242,7 +242,7 @@ public void shouldCaptureApplicationAndRecordMetadata() { final Properties config = mkProperties( mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "testMetadata"), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092") ) ); @@ -336,7 +336,7 @@ public void process(final Record record) {} public void fullConstructorShouldSetAllExpectedAttributes() { final Properties config = new Properties(); config.put(StreamsConfig.APPLICATION_ID_CONFIG, "testFullConstructor"); - config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"); config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.StringSerde.class.getName()); config.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.LongSerde.class.getName()); diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java index 521a39d692b9f..29e19b4ce0ac9 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/test/MockProcessorContextStateStoreTest.java @@ -166,7 +166,7 @@ public void shouldEitherInitOrThrow(final StoreBuilder builder, final MockProcessorContext context = new MockProcessorContext<>( mkProperties(mkMap( mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, ""), - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "") + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092") )), new TaskId(0, 0), stateDir From d80d02aa78952c8d57c6460f227d6a16369e0fac Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 16:50:52 +0800 Subject: [PATCH 33/78] MirrorClientConfig and test --- .../org/apache/kafka/connect/mirror/MirrorClientConfig.java | 2 +- .../apache/kafka/connect/mirror/MirrorMakerConfigTest.java | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) 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 acfbe843b95ed..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 @@ -134,7 +134,7 @@ private Map clientConfig(String prefix) { static final ConfigDef CONFIG_DEF = new ConfigDef() .define(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, - Type.STRING, + Type.LIST, NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java index d728b29f0ba52..ddd22b0b8ad8e 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorMakerConfigTest.java @@ -28,6 +28,7 @@ import org.junit.jupiter.api.Test; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -104,7 +105,7 @@ public void testClientConfigProperties() { "replication.policy.separator is picked up in MirrorClientConfig"); assertEquals("b__topic1", aClientConfig.replicationPolicy().formatRemoteTopic("b", "topic1"), "replication.policy.separator is honored"); - assertEquals(clusterABootstrap, aClientConfig.adminConfig().get("bootstrap.servers"), + assertEquals(Arrays.stream(clusterABootstrap.split(",")).map(String::trim).toList(), aClientConfig.adminConfig().get("bootstrap.servers"), "client configs include bootstrap.servers"); try (ForwardingAdmin forwardingAdmin = aClientConfig.forwardingAdmin(aClientConfig.adminConfig())) { assertEquals(ForwardingAdmin.class.getName(), forwardingAdmin.getClass().getName(), @@ -178,7 +179,7 @@ public void testIncludesTopicFilterProperties() { SourceAndTarget sourceAndTarget = new SourceAndTarget("source", "target"); Map connectorProps = mirrorConfig.connectorBaseConfig(sourceAndTarget, MirrorSourceConnector.class); - DefaultTopicFilter.TopicFilterConfig filterConfig = + DefaultTopicFilter.TopicFilterConfig filterConfig = new DefaultTopicFilter.TopicFilterConfig(connectorProps); assertEquals(List.of("topic1", "topic2"), filterConfig.getList("topics"), "source->target.topics should be passed through to TopicFilters."); From 8f6e3b70b1130c7db0e5d08be972cc47ca966caf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 16:53:44 +0800 Subject: [PATCH 34/78] fix StateDirectoryTest and TimeOrderedKeyValueBufferTest --- .../streams/processor/internals/StateDirectoryTest.java | 6 +++--- .../state/internals/TimeOrderedKeyValueBufferTest.java | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index f850093d389d4..3b795310b3615 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -139,7 +139,7 @@ public void shouldHaveSecurePermissions() { assertPermissions(stateDir); assertPermissions(appDir); } - + private void assertPermissions(final File file) { final Path path = file.toPath(); if (path.getFileSystem().supportedFileAttributeViews().contains("posix")) { @@ -630,7 +630,7 @@ public void shouldLogTempDirMessage() { new StateDirectory( new StreamsConfig( mkMap( - mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""), + mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock-localhost:9092"), mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, "") ) ), @@ -1004,4 +1004,4 @@ public void run() { } } } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java index 56c19c985299a..0a65d47d207f2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/TimeOrderedKeyValueBufferTest.java @@ -98,7 +98,7 @@ private void setup(final String testName, final Function bufferSuppli private static MockInternalProcessorContext makeContext() { final Properties properties = new Properties(); properties.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID); - properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, ""); + properties.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "mock:localhost:9092"); final TaskId taskId = new TaskId(0, 0); From 29a1dd3343b4efae0b04bf87e075a4ba66371fcf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 17:02:09 +0800 Subject: [PATCH 35/78] fix ConsumerConfigTest, ConsumerPartitionAssignorTest, GroupRebalanceConfigTest, KafkaConsumerTest --- .../kafka/clients/GroupRebalanceConfigTest.java | 11 +++++++---- .../kafka/clients/consumer/ConsumerConfigTest.java | 13 ++++++++++++- .../consumer/ConsumerPartitionAssignorTest.java | 1 + .../kafka/clients/consumer/KafkaConsumerTest.java | 1 + 4 files changed, 21 insertions(+), 5 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java index 7f02204d4c4e0..41bd0cff42bf1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/GroupRebalanceConfigTest.java @@ -34,13 +34,14 @@ void testRackIdIsEmptyIfNoDefined(GroupRebalanceConfig.ProtocolType protocolType GroupRebalanceConfig groupRebalanceConfig = new GroupRebalanceConfig( new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer" + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); assertTrue(groupRebalanceConfig.rackId.isEmpty()); } - + @ParameterizedTest @EnumSource(value = GroupRebalanceConfig.ProtocolType.class, names = {"CONSUMER", "SHARE"}) void testRackIdIsEmptyIfValueIsEmptyString(GroupRebalanceConfig.ProtocolType protocolType) { @@ -48,7 +49,8 @@ void testRackIdIsEmptyIfValueIsEmptyString(GroupRebalanceConfig.ProtocolType pro new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.CLIENT_RACK_CONFIG, "" + ConsumerConfig.CLIENT_RACK_CONFIG, "", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); @@ -62,7 +64,8 @@ void testRackIdIsNotEmptyIfDefined(GroupRebalanceConfig.ProtocolType protocolTyp new ConsumerConfig(Map.of( ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.CLIENT_RACK_CONFIG, "rack1" + ConsumerConfig.CLIENT_RACK_CONFIG, "rack1", + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" )), protocolType ); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index 2fa5515fb4073..0065f5f4dc50c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -56,6 +56,7 @@ public class ConsumerConfigTest { public void setUp() { properties.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClassName); properties.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClassName); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); } @Test @@ -144,6 +145,7 @@ public void testInvalidGroupInstanceId() { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, ""); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG)); } @@ -154,6 +156,7 @@ public void testInvalidSecurityProtocol() { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -164,6 +167,7 @@ public void testCaseInsensitiveSecurityProtocol() { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(saslSslLowerCase, consumerConfig.originals().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); @@ -174,6 +178,7 @@ public void testDefaultConsumerGroupConfig() { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals("classic", consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); assertNull(consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); @@ -188,6 +193,7 @@ public void testRemoteAssignorConfig() { configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, remoteAssignorName); configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, protocol); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(protocol, consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); assertEquals(remoteAssignorName, consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); @@ -200,6 +206,7 @@ public void testRemoteAssignorWithClassicGroupProtocol() { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, remoteAssignorName); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException exception = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(exception.getMessage().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG + " cannot be set when " + ConsumerConfig.GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CLASSIC.name())); } @@ -209,6 +216,7 @@ public void testDefaultMetadataRecoveryStrategy() { Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); assertEquals(MetadataRecoveryStrategy.REBOOTSTRAP.name, consumerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -219,6 +227,7 @@ public void testInvalidMetadataRecoveryStrategy() { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -230,6 +239,7 @@ public void testProtocolConfigValidation(String protocol, boolean isValid) { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, protocol); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); if (isValid) { ConsumerConfig config = new ConsumerConfig(configs); assertEquals(protocol, config.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); @@ -250,10 +260,11 @@ private void testUnsupportedConfigsWithConsumerGroupProtocol(String configName, ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass, ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass, ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CONSUMER.name(), + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092", configName, value ); ConfigException exception = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); - assertEquals(configName + " cannot be set when " + + assertEquals(configName + " cannot be set when " + ConsumerConfig.GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CONSUMER.name(), exception.getMessage()); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java index b4f649de579ae..c08c7766ec1c7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignorTest.java @@ -151,6 +151,7 @@ private ConsumerConfig initConsumerConfigWithClassTypes(List classTypes) props.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classTypes); props.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name()); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ConsumerConfig(props); } } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 4069b9d25c1c1..82d080d682078 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -3089,6 +3089,7 @@ private ConsumerConfig newConsumerConfig(GroupProtocol groupProtocol, configs.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, retryBackoffMs); configs.put(ConsumerConfig.THROW_ON_FETCH_STABLE_OFFSET_UNSUPPORTED, throwOnStableOffsetNotSupported); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); groupInstanceId.ifPresent(gi -> configs.put(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, gi)); return new ConsumerConfig(configs); From 6cfad77e0eceebf1a224516d568a18ceca83c2d4 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 17:12:54 +0800 Subject: [PATCH 36/78] fix other tests --- .../consumer/KafkaShareConsumerMetricsTest.java | 1 + .../clients/consumer/KafkaShareConsumerTest.java | 3 ++- .../internals/CommitRequestManagerTest.java | 15 ++++++++------- .../internals/FetchRequestManagerTest.java | 1 + .../internals/ShareConsumeRequestManagerTest.java | 5 +++-- .../StreamsGroupHeartbeatRequestManagerTest.java | 3 ++- .../TopicMetadataRequestManagerTest.java | 1 + .../clients/producer/ProducerConfigTest.java | 14 +++++++++++--- 8 files changed, 29 insertions(+), 14 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java index ad46bf0887ebc..a5417c3e00fd1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerMetricsTest.java @@ -367,6 +367,7 @@ private ShareConsumerConfig newConsumerConfig(String groupId, configs.put(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG, retryBackoffMaxMs); configs.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, retryBackoffMs); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializer.getClass()); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ShareConsumerConfig(configs); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java index f4b383df67c74..1a6d76dbabfc3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaShareConsumerTest.java @@ -307,6 +307,7 @@ private ShareConsumerConfig newConsumerConfig(String clientId) { configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); configs.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, batchSize); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ShareConsumerConfig(configs); } @@ -414,4 +415,4 @@ private ShareAcknowledgeResponse shareAcknowledgeResponse(TopicIdPartition tip) .setResponses(new ShareAcknowledgeResponseData.ShareAcknowledgeTopicResponseCollection(List.of(topicResponse).iterator())) ); } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java index d4ceeedde56b1..71b21eca314fd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/CommitRequestManagerTest.java @@ -718,7 +718,7 @@ public void testOffsetFetchRequestErroredRequests(final Errors error) { @ParameterizedTest @MethodSource("offsetFetchExceptionSupplier") - public void testOffsetFetchRequestTimeoutRequests(final Errors error, + public void testOffsetFetchRequestTimeoutRequests(final Errors error, final Class expectedExceptionClass) { CommitRequestManager commitRequestManager = create(true, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -914,7 +914,7 @@ public void testOffsetCommitSyncTimeoutNotReturnedOnPollAndFails() { @ParameterizedTest @MethodSource("offsetCommitExceptionSupplier") public void testOffsetCommitSyncFailedWithRetriableThrowsTimeoutWhenRetryTimeExpires( - final Errors error, + final Errors error, final Class expectedExceptionClass) { CommitRequestManager commitRequestManager = create(false, 100); when(coordinatorRequestManager.coordinator()).thenReturn(Optional.of(mockedNode)); @@ -1336,7 +1336,7 @@ private static Stream offsetCommitExceptionSupplier() { Arguments.of(Errors.COORDINATOR_NOT_AVAILABLE, TimeoutException.class), Arguments.of(Errors.REQUEST_TIMED_OUT, TimeoutException.class), Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, TimeoutException.class), - + // Non-retriable errors should result in their specific exceptions Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, GroupAuthorizationException.class), Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, OffsetMetadataTooLarge.class), @@ -1344,7 +1344,7 @@ private static Stream offsetCommitExceptionSupplier() { Arguments.of(Errors.TOPIC_AUTHORIZATION_FAILED, TopicAuthorizationException.class), Arguments.of(Errors.UNKNOWN_MEMBER_ID, CommitFailedException.class), Arguments.of(Errors.STALE_MEMBER_EPOCH, CommitFailedException.class), - + // Generic errors should result in KafkaException Arguments.of(Errors.UNKNOWN_SERVER_ERROR, KafkaException.class)); } @@ -1361,7 +1361,7 @@ private static Stream offsetFetchExceptionSupplier() { Arguments.of(Errors.REQUEST_TIMED_OUT, TimeoutException.class), Arguments.of(Errors.UNSTABLE_OFFSET_COMMIT, TimeoutException.class), Arguments.of(Errors.UNKNOWN_TOPIC_OR_PARTITION, TimeoutException.class), - + // Non-retriable errors should result in their specific exceptions Arguments.of(Errors.GROUP_AUTHORIZATION_FAILED, GroupAuthorizationException.class), Arguments.of(Errors.OFFSET_METADATA_TOO_LARGE, KafkaException.class), @@ -1372,7 +1372,7 @@ private static Stream offsetFetchExceptionSupplier() { // Adding STALE_MEMBER_EPOCH as non-retriable here because it is only retried if a new // member epoch is received. Tested separately. Arguments.of(Errors.STALE_MEMBER_EPOCH, StaleMemberEpochException.class), - + // Generic errors should result in KafkaException Arguments.of(Errors.UNKNOWN_SERVER_ERROR, KafkaException.class)); } @@ -1484,7 +1484,7 @@ public void testPollWithFatalErrorShouldFailAllUnsentRequests() { assertEmptyPendingRequests(commitRequestManager); } - + private static void assertEmptyPendingRequests(CommitRequestManager commitRequestManager) { assertTrue(commitRequestManager.pendingRequests.inflightOffsetFetches.isEmpty()); assertTrue(commitRequestManager.pendingRequests.unsentOffsetFetches.isEmpty()); @@ -1561,6 +1561,7 @@ private List assertPoll( private CommitRequestManager create(final boolean autoCommitEnabled, final long autoCommitInterval) { props.setProperty(AUTO_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(autoCommitInterval)); props.setProperty(ENABLE_AUTO_COMMIT_CONFIG, String.valueOf(autoCommitEnabled)); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); if (autoCommitEnabled) props.setProperty(GROUP_ID_CONFIG, TestUtils.randomString(10)); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java index 0f83e28d9e4d8..f806ab65b6b65 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java @@ -4125,6 +4125,7 @@ private void buildDependencies(MetricConfig metricConfig, properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(requestTimeoutMs)); properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate(time, config, logContext, client, metadata, backgroundEventHandler, true)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java index 68da71d7767c1..a4268b7eca0a7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ShareConsumeRequestManagerTest.java @@ -2477,7 +2477,7 @@ public void testCloseInternalClosesShareFetchMetricsManager() throws Exception { // Verify that sensors exist before closing for (String sensorName : sensorNames) { - assertNotNull(metrics.getSensor(sensorName), + assertNotNull(metrics.getSensor(sensorName), "Sensor " + sensorName + " should exist before closing"); } @@ -2486,7 +2486,7 @@ public void testCloseInternalClosesShareFetchMetricsManager() throws Exception { // Verify that all sensors are removed after closing for (String sensorName : sensorNames) { - assertNull(metrics.getSensor(sensorName), + assertNull(metrics.getSensor(sensorName), "Sensor " + sensorName + " should be removed after closing"); } } @@ -2683,6 +2683,7 @@ private void buildDependencies(MetricConfig metricConfig, properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.setProperty(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG, String.valueOf(requestTimeoutMs)); properties.setProperty(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, String.valueOf(retryBackoffMs)); + properties.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConsumerConfig config = new ConsumerConfig(properties); networkClientDelegate = spy(new TestableNetworkClientDelegate( time, config, logContext, client, metadata, diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java index 9839f3b221017..15e2387556232 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/StreamsGroupHeartbeatRequestManagerTest.java @@ -1555,6 +1555,7 @@ private static ConsumerConfig config() { prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); prop.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, String.valueOf(DEFAULT_MAX_POLL_INTERVAL_MS)); + prop.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new ConsumerConfig(prop); } @@ -1623,4 +1624,4 @@ private static void assertTaskIdsEquals(final List new ProducerConfig(configs)); } @@ -97,6 +99,7 @@ public void testInvalidSecurityProtocol() { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -106,6 +109,7 @@ public void testDefaultMetadataRecoveryStrategy() { Map configs = new HashMap<>(); configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ProducerConfig producerConfig = new ProducerConfig(configs); assertEquals(MetadataRecoveryStrategy.REBOOTSTRAP.name, producerConfig.getString(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -116,6 +120,7 @@ public void testInvalidMetadataRecoveryStrategy() { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG, "abc"); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(CommonClientConfigs.METADATA_RECOVERY_STRATEGY_CONFIG)); } @@ -127,6 +132,7 @@ public void testCaseInsensitiveSecurityProtocol() { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, saslSslLowerCase); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); final ProducerConfig producerConfig = new ProducerConfig(configs); assertEquals(saslSslLowerCase, producerConfig.originals().get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)); } @@ -138,6 +144,7 @@ void testUpperboundCheckOfEnableIdempotence() { configs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, keySerializerClass); configs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass); configs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, inFlightConnection); + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); ConfigException configException = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertEquals("To use the idempotent producer, " + ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION + " must be set to at most 5. Current value is " + inFlightConnection + ".", configException.getMessage()); @@ -155,15 +162,16 @@ void testTwoPhaseCommitIncompatibleWithTransactionTimeout() { configs.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, "test-txn-id"); configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, true); configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); - + configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + ConfigException ce = assertThrows(ConfigException.class, () -> new ProducerConfig(configs)); assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG)); assertTrue(ce.getMessage().contains(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG)); - + // Verify that setting one but not the other is valid configs.remove(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG); assertDoesNotThrow(() -> new ProducerConfig(configs)); - + configs.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, 60000); configs.put(ProducerConfig.TRANSACTION_TWO_PHASE_COMMIT_ENABLE_CONFIG, false); assertDoesNotThrow(() -> new ProducerConfig(configs)); From 4b7cd4e0863bd12523c82cb45b5f094dc044431d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 20:11:56 +0800 Subject: [PATCH 37/78] fix NetworkClientDelegateTest --- .../clients/consumer/internals/NetworkClientDelegateTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java index 4ff967e1f021a..e10482c31ceda 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/NetworkClientDelegateTest.java @@ -49,6 +49,7 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.LinkedBlockingQueue; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG; @@ -290,6 +291,7 @@ public NetworkClientDelegate newNetworkClientDelegate(boolean notifyMetadataErro properties.put(VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); properties.put(GROUP_ID_CONFIG, GROUP_ID); properties.put(REQUEST_TIMEOUT_MS_CONFIG, REQUEST_TIMEOUT_MS); + properties.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return new NetworkClientDelegate(time, new ConsumerConfig(properties), logContext, From c66ec10c6a515f667568edcc60ebc2165b879113 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 20:59:49 +0800 Subject: [PATCH 38/78] update WorkerConfig --- .../apache/kafka/connect/mirror/DefaultGroupFilter.java | 2 ++ .../kafka/connect/mirror/MirrorCheckpointTaskConfig.java | 6 +----- .../kafka/connect/mirror/MirrorSourceTaskConfig.java | 3 --- .../org/apache/kafka/connect/runtime/WorkerConfig.java | 5 +++-- .../kafka/connect/runtime/rest/RestServerConfig.java | 7 +++---- 5 files changed, 9 insertions(+), 14 deletions(-) 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 f88ed4e704661..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,11 +65,13 @@ static class GroupFilterConfig extends AbstractConfig { .define(GROUPS_INCLUDE_CONFIG, Type.LIST, GROUPS_INCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, GROUPS_INCLUDE_DOC) .define(GROUPS_EXCLUDE_CONFIG, Type.LIST, GROUPS_EXCLUDE_DEFAULT, + ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.HIGH, GROUPS_EXCLUDE_DOC); diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java index c482667379d8c..455eedcaaa05e 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 @@ -32,11 +32,7 @@ public MirrorCheckpointTaskConfig(Map props) { } Set taskConsumerGroups() { - List fields = getList(TASK_CONSUMER_GROUPS); - if (fields.isEmpty()) { - return Set.of(); - } - return new HashSet<>(fields); + return new HashSet<>(getList(TASK_CONSUMER_GROUPS)); } MirrorCheckpointMetrics metrics() { 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 0986d76cf717a..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 @@ -34,9 +34,6 @@ public MirrorSourceTaskConfig(Map props) { Set taskTopicPartitions() { List fields = getList(TASK_TOPIC_PARTITIONS); - if (fields.isEmpty()) { - return Set.of(); - } return fields.stream() .map(MirrorUtils::decodeTopicPartition) .collect(Collectors.toSet()); 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 d48ac2583da9c..8e3a4152564e0 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 @@ -65,7 +65,8 @@ public class WorkerConfig extends AbstractConfig { + "While the order of servers in the list does not matter, we recommend including more than one server to ensure resilience if any servers are down. " + "This list does not need to contain the entire set of brokers, as Kafka clients automatically manage and update connections to the cluster efficiently. " + "This list must be in the form host1:port1,host2:port2,...."; - public static final List BOOTSTRAP_SERVERS_DEFAULT = List.of("localhost:9092"); + @Deprecated(since = "4.2") + public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG; public static final String CLIENT_DNS_LOOKUP_DOC = CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC; @@ -199,7 +200,7 @@ public class WorkerConfig extends AbstractConfig { */ protected static ConfigDef baseConfigDef() { ConfigDef result = new ConfigDef() - .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, BOOTSTRAP_SERVERS_DEFAULT, + .define(BOOTSTRAP_SERVERS_CONFIG, Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), Importance.HIGH, BOOTSTRAP_SERVERS_DOC) .define(CLIENT_DNS_LOOKUP_CONFIG, 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 b5255322666fa..2bcf117f7d594 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 @@ -140,13 +140,12 @@ public abstract class RestServerConfig extends AbstractConfig { public static void addPublicConfig(ConfigDef configDef) { addInternalConfig(configDef); configDef - .define( - REST_EXTENSION_CLASSES_CONFIG, + .define(REST_EXTENSION_CLASSES_CONFIG, ConfigDef.Type.LIST, List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), - ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC - ).define(ADMIN_LISTENERS_CONFIG, + ConfigDef.Importance.LOW, REST_EXTENSION_CLASSES_DOC) + .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, new AdminListenersValidator(), From d2a048b1d7a759905fa399c2ac1155d8c498e071 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 12 Aug 2025 21:38:52 +0800 Subject: [PATCH 39/78] spotlessApply --- .../apache/kafka/connect/mirror/MirrorCheckpointTaskConfig.java | 1 - 1 file changed, 1 deletion(-) 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 455eedcaaa05e..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 @@ -19,7 +19,6 @@ import org.apache.kafka.common.config.ConfigDef; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; From 9c7fcee860dfddf84a099440c4b925da329321c3 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 13 Aug 2025 12:41:56 +0800 Subject: [PATCH 40/78] fix the WorkerConfig related test --- .../org/apache/kafka/connect/runtime/WorkerConfig.java | 2 -- .../connect/runtime/AbstractWorkerSourceTaskTest.java | 1 + .../apache/kafka/connect/runtime/ConnectMetricsTest.java | 8 +++++--- .../kafka/connect/runtime/ErrorHandlingTaskTest.java | 1 + .../connect/runtime/ExactlyOnceWorkerSourceTaskTest.java | 3 ++- .../apache/kafka/connect/runtime/MockConnectMetrics.java | 1 + .../connect/runtime/SourceTaskOffsetCommitterTest.java | 1 + .../apache/kafka/connect/runtime/WorkerSinkTaskTest.java | 1 + .../kafka/connect/runtime/WorkerSinkTaskThreadedTest.java | 5 +++-- .../kafka/connect/runtime/WorkerSourceTaskTest.java | 1 + .../java/org/apache/kafka/connect/runtime/WorkerTest.java | 3 ++- .../runtime/distributed/WorkerGroupMemberTest.java | 2 ++ .../runtime/errors/RetryWithToleranceOperatorTest.java | 1 + .../kafka/connect/runtime/isolation/MultiVersionTest.java | 1 + .../kafka/connect/runtime/isolation/PluginsTest.java | 1 + .../connect/runtime/standalone/StandaloneConfigTest.java | 1 + .../kafka/connect/storage/FileOffsetBackingStoreTest.java | 2 ++ 17 files changed, 26 insertions(+), 9 deletions(-) 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 8e3a4152564e0..26061e292cdc2 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 @@ -65,8 +65,6 @@ public class WorkerConfig extends AbstractConfig { + "While the order of servers in the list does not matter, we recommend including more than one server to ensure resilience if any servers are down. " + "This list does not need to contain the entire set of brokers, as Kafka clients automatically manage and update connections to the cluster efficiently. " + "This list must be in the form host1:port1,host2:port2,...."; - @Deprecated(since = "4.2") - public static final String BOOTSTRAP_SERVERS_DEFAULT = "localhost:9092"; public static final String CLIENT_DNS_LOOKUP_CONFIG = CommonClientConfigs.CLIENT_DNS_LOOKUP_CONFIG; public static final String CLIENT_DNS_LOOKUP_DOC = CommonClientConfigs.CLIENT_DNS_LOOKUP_DOC; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java index 97b8edf7605f7..5b80ae32cfee2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractWorkerSourceTaskTest.java @@ -171,6 +171,7 @@ public void setup() { private Map workerProps() { Map props = new HashMap<>(); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java index b473315758774..8ba0316e20c0d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectMetricsTest.java @@ -62,15 +62,17 @@ public class ConnectMetricsTest { private static final Map DEFAULT_WORKER_CONFIG = Map.of( WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter", - WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); + WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter", + WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092" + ); private static final ConnectorTaskId CONNECTOR_TASK_ID = new ConnectorTaskId("connector", 0); private static final LinkedHashMap TAGS = new LinkedHashMap<>(); private ConnectMetrics metrics; - + static { TAGS.put("t1", "v1"); } - + @BeforeEach public void setUp() { metrics = new ConnectMetrics("worker1", new WorkerConfig(WorkerConfig.baseConfigDef(), DEFAULT_WORKER_CONFIG), new MockTime(), "cluster-1"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index 340585f0fd66e..b14851ed125a1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -187,6 +187,7 @@ public void setup(boolean enableTopicCreation) { workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); workerConfig = new StandaloneConfig(workerProps); sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java index f45ec27b46e2d..684891d255e38 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ExactlyOnceWorkerSourceTaskTest.java @@ -232,6 +232,7 @@ public void teardown() throws Exception { private Map workerProps() { Map props = new HashMap<>(); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); @@ -638,7 +639,7 @@ public void testIntervalBasedCommit(boolean enableTopicCreation) throws Exceptio time.sleep(commitInterval * 2); awaitPolls(2); - assertEquals(2, flushCount(), + assertEquals(2, flushCount(), "Two flushes should have taken place after offset commit interval has elapsed again"); awaitShutdown(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java index c5f9f8314d9ef..b6548651418d6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/MockConnectMetrics.java @@ -49,6 +49,7 @@ public class MockConnectMetrics extends ConnectMetrics { DEFAULT_WORKER_CONFIG.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); DEFAULT_WORKER_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); DEFAULT_WORKER_CONFIG.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + DEFAULT_WORKER_CONFIG.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); } public MockConnectMetrics() { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java index 1ebb4e053196a..8c6e67fdf197c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java @@ -73,6 +73,7 @@ public class SourceTaskOffsetCommitterTest { @BeforeEach public void setup() { Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index abde7340e8dd7..f3448a72af8c6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -183,6 +183,7 @@ public class WorkerSinkTaskTest { public void setUp() { time = new MockTime(); Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 0693f06dfe4d4..f44915c8b5fad 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -174,6 +174,7 @@ public void setup() { workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); WorkerConfig workerConfig = new StandaloneConfig(workerProps); Plugin keyConverterPlugin = metrics.wrap(keyConverter, taskId, true); Plugin valueConverterPlugin = metrics.wrap(valueConverter, taskId, false); @@ -651,7 +652,7 @@ private void expectPreCommit(ExpectOffsetCommitCommand... commands) { @Override public Object answer(InvocationOnMock invocation) { ExpectOffsetCommitCommand commitCommand = commands[index++]; - // All assigned partitions will have offsets committed, but we've only processed messages/updated + // All assigned partitions will have offsets committed, but we've only processed messages/updated // offsets for one final Map offsetsToCommit = offsetsToCommitFn.apply(commitCommand.expectedMessages); @@ -664,7 +665,7 @@ public Object answer(InvocationOnMock invocation) { } }).when(sinkTask).preCommit(anyMap()); } - + private void expectOffsetCommit(ExpectOffsetCommitCommand... commands) { doAnswer(new Answer<>() { int index = 0; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index d2fd923fdb93b..287f6ba412950 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -204,6 +204,7 @@ private Map workerProps(boolean enableTopicCreation) { props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); props.put("offset.storage.file.filename", "/tmp/connect.offsets"); props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); return props; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 38bcab1b59469..bbdee58181c05 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -250,6 +250,7 @@ public void setup(boolean enableTopicCreation) { .strictness(Strictness.STRICT_STUBS) .startMocking(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); @@ -2864,7 +2865,7 @@ private void testStartTaskWithTooManyTaskConfigs(boolean enforced) { ArgumentCaptor failureCaptor = ArgumentCaptor.forClass(Throwable.class); verify(taskStatusListener, times(1)).onFailure(eq(TASK_ID), failureCaptor.capture()); - assertInstanceOf(TooManyTasksException.class, failureCaptor.getValue(), + assertInstanceOf(TooManyTasksException.class, failureCaptor.getValue(), "Expected task start exception to be TooManyTasksException, but was " + failureCaptor.getValue().getClass() + " instead"); tasksMaxExceededMessage = failureCaptor.getValue().getMessage(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java index fad39d84129dc..4886431869c1e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMemberTest.java @@ -59,6 +59,7 @@ public class WorkerGroupMemberTest { public void testMetrics() throws Exception { WorkerGroupMember member; Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); @@ -102,6 +103,7 @@ public void testMetrics() throws Exception { public void testDisableJmxReporter() { WorkerGroupMember member; Map workerProps = new HashMap<>(); + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); workerProps.put("group.id", "group-1"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java index ec3ecefea96da..cc35559efd05f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/errors/RetryWithToleranceOperatorTest.java @@ -93,6 +93,7 @@ public class RetryWithToleranceOperatorTest { put(CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG, Sensor.RecordingLevel.INFO.toString()); // define required properties + put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); }}; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java index b770de74b80b3..b27444ab8ca8b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/MultiVersionTest.java @@ -267,6 +267,7 @@ public void testVersionedConverter() { converterConfig.put(WorkerConfig.VALUE_CONVERTER_VERSION, "2.3.0"); converterConfig.put(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG, VersionedPluginBuilder.VersionedTestPlugin.HEADER_CONVERTER.className()); converterConfig.put(WorkerConfig.HEADER_CONVERTER_VERSION, "4.3.0"); + converterConfig.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); AbstractConfig config; try (LoaderSwap swap = plugins.safeLoaderSwapper().apply(plugins.delegatingLoader())) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java index ca4c29931d088..d423eca05a698 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/isolation/PluginsTest.java @@ -93,6 +93,7 @@ public void setup() { pluginProps.put(WorkerConfig.PLUGIN_PATH_CONFIG, TestPlugins.pluginPathJoined()); plugins = new Plugins(pluginProps); props = new HashMap<>(pluginProps); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); props.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); props.put("key.converter." + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "true"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java index 4d8c25932fe42..c4d52fe4b4e5b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneConfigTest.java @@ -48,6 +48,7 @@ private Map sslProps() { private Map baseWorkerProps() { return new HashMap<>() { { + put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, "/tmp/foo"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java index 204fcc283bd94..195064a83b390 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/FileOffsetBackingStoreTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.connect.data.SchemaAndValue; import org.apache.kafka.connect.json.JsonConverter; import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.util.Callback; @@ -86,6 +87,7 @@ public void setup() { props.put(StandaloneConfig.OFFSET_STORAGE_FILE_FILENAME_CONFIG, tempFile.getAbsolutePath()); props.put(StandaloneConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); props.put(StandaloneConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter"); + props.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); config = new StandaloneConfig(props); store.configure(config); store.start(); From 58cea0f16699100dd0a6cc70b341c387444590bf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 13 Aug 2025 20:54:36 +0800 Subject: [PATCH 41/78] update the null checker for config.providers and consumer assignors --- .../kafka/clients/consumer/ConsumerPartitionAssignor.java | 3 --- .../java/org/apache/kafka/common/config/AbstractConfig.java | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java index 20f2551ba6bc2..45cb505c744e5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPartitionAssignor.java @@ -291,9 +291,6 @@ static List getAssignorInstances(List assigno // a map to store assignor name -> assignor class name Map assignorNameMap = new HashMap<>(); - if (assignorClasses == null) - return assignors; - for (Object klass : assignorClasses) { // first try to get the class if passed in as a string if (klass instanceof String) { diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index e271cd99c4cd0..3029ca0e73d0e 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 @@ -605,7 +605,7 @@ private Map instantiateConfigProviders( ) { final String configProviders = indirectConfigs.get(CONFIG_PROVIDERS_CONFIG); - if (configProviders == null || configProviders.isEmpty()) { + if (configProviders.isEmpty()) { return Collections.emptyMap(); } From b281ee21af2fff94d24794dfd028de0a23dcb35d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 13 Aug 2025 22:27:46 +0800 Subject: [PATCH 42/78] revert config.providers --- .../java/org/apache/kafka/common/config/AbstractConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3029ca0e73d0e..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 @@ -605,7 +605,7 @@ private Map instantiateConfigProviders( ) { final String configProviders = indirectConfigs.get(CONFIG_PROVIDERS_CONFIG); - if (configProviders.isEmpty()) { + if (configProviders == null || configProviders.isEmpty()) { return Collections.emptyMap(); } From 381884873ca3d2a96bad468ec745bd035de06e63 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 20 Aug 2025 21:47:24 +0800 Subject: [PATCH 43/78] addressed some comments --- .../org/apache/kafka/common/config/ConfigDef.java | 11 ++++++----- .../org/apache/kafka/common/config/SslConfigs.java | 4 +++- .../org/apache/kafka/common/config/TopicConfig.java | 4 +++- .../main/scala/kafka/metrics/KafkaMetricsConfig.scala | 4 ++-- core/src/main/scala/kafka/server/KafkaConfig.scala | 8 ++++---- .../org/apache/kafka/network/SocketServerConfigs.java | 9 ++++----- .../kafka/server/config/AbstractKafkaConfig.java | 4 ++-- 7 files changed, 24 insertions(+), 20 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 a42477215033e..cb394f6545c0e 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 @@ -1024,7 +1024,7 @@ public static ValidList in(String... validStrings) { } public static ValidList in(boolean isEmptyAllowed, String... validStrings) { - if (validStrings.length == 0) { + if (!isEmptyAllowed && validStrings.length == 0) { throw new IllegalArgumentException("Valid strings list cannot be empty for inNonEmpty validator"); } return new ValidList(List.of(validStrings), isEmptyAllowed, false); @@ -1032,10 +1032,11 @@ public static ValidList in(boolean isEmptyAllowed, String... validStrings) { @Override public void ensureValid(final String name, final Object value) { - if (value == null && isNullAllowed) { - return; - } else if (value == null) { - throw new ConfigException("Configuration '" + name + "' values must not be null."); + if (value == null) { + if (isNullAllowed) + return; + else + throw new ConfigException("Configuration '" + name + "' values must not be null."); } @SuppressWarnings("unchecked") 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 dda04651be14c..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,7 +50,9 @@ public class SslConfigs { public static final String SSL_ENABLED_PROTOCOLS_DOC = "The list of protocols enabled for SSL connections. " + "The default is 'TLSv1.2,TLSv1.3'. This means that clients and servers will prefer TLSv1.3 if both support it " + "and fallback to TLSv1.2 otherwise (assuming both support at least TLSv1.2). This default should be fine for most use " - + "cases. Also see the config documentation for ssl.protocol to understand how it can impact the TLS version negotiation behavior."; + + "cases. If this configuration is set to an empty list, Kafka will use the protocols enabled by default in the underlying SSLEngine, " + + "which may include additional protocols depending on the JVM version. " + + "Also see the config documentation for ssl.protocol to understand how it can impact the TLS version negotiation behavior."; public static final String DEFAULT_SSL_ENABLED_PROTOCOLS = "TLSv1.2,TLSv1.3"; public static final String SSL_KEYSTORE_TYPE_CONFIG = "ssl.keystore.type"; diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index 958df8abc311d..c9a284e3578bf 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -159,7 +159,9 @@ public class TopicConfig { "log compaction, which retains the latest value for each key. " + "It is also possible to specify both policies in a comma-separated list (e.g. \"delete,compact\"). " + "In this case, old segments will be discarded per the retention time and size configuration, " + - "while retained segments will be compacted."; + "while retained segments will be compacted." + + "An empty list means infinite retention - no cleanup policies will be applied and log segments " + + "will be retained indefinitely."; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = "unclean.leader.election.enable"; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas " + diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index c3eb15082175d..fe1050222b12b 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -33,8 +33,8 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * Comma-separated list of reporter types. These classes should be on the * classpath and will be instantiated at run-time. */ - val reporters: Seq[String] = Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, - String.join("", MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT))).asScala + val reporters: Seq[String] = (if (props.containsKey(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG)) + Csv.parseCsvList(props.getString(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG)) else MetricConfigs.KAFKA_METRIC_REPORTER_CLASSES_DEFAULT).asScala /** * The metrics polling interval (in seconds). diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index fcf9f425df669..8ac4f5b333c5d 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -451,9 +451,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) } def effectiveAdvertisedControllerListeners: Seq[Endpoint] = { - val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) + val advertisedListenersProp = getList(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val controllerAdvertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(Csv.parseCsvList(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) .filter(l => controllerListenerNames.contains(l.listener)) } else { Seq.empty @@ -481,9 +481,9 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _]) def effectiveAdvertisedBrokerListeners: Seq[Endpoint] = { // Use advertised listeners if defined, fallback to listeners otherwise - val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) + val advertisedListenersProp = getList(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListeners = if (advertisedListenersProp != null) { - CoreUtils.listenerListToEndPoints(Csv.parseCsvList(advertisedListenersProp), effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) + CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) } else { listeners } 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 7e59714477839..0f136f2e9f236 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -156,7 +156,7 @@ public class SocketServerConfigs { 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, STRING, null, HIGH, ADVERTISED_LISTENERS_DOC) + .define(ADVERTISED_LISTENERS_CONFIG, LIST, null, 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) @@ -204,13 +204,12 @@ public static List listenerListToEndPoints( ) { List results = new ArrayList<>(); for (String entry : input) { - String trimEntry = entry.trim(); - if (trimEntry.isEmpty()) { + if (entry.isEmpty()) { continue; } - Matcher matcher = URI_PARSE_REGEXP.matcher(trimEntry); + Matcher matcher = URI_PARSE_REGEXP.matcher(entry); if (!matcher.matches()) { - throw new KafkaException("Unable to parse " + trimEntry + " to a broker endpoint"); + throw new KafkaException("Unable to parse " + entry + " to a broker endpoint"); } ListenerName listenerName = ListenerName.normalised(matcher.group(1)); String host = matcher.group(2); diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 231c1cbf86776..3e4e8ab5c52f8 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -39,12 +39,12 @@ import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; -import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * During moving {@link kafka.server.KafkaConfig} out of core AbstractKafkaConfig will be the future KafkaConfig @@ -81,7 +81,7 @@ public AbstractKafkaConfig(ConfigDef definition, Map originals, Map logDirs() { return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)) - .orElse(Arrays.stream(getString(ServerLogConfigs.LOG_DIR_CONFIG).split(",")) + .orElse(Stream.of(getString(ServerLogConfigs.LOG_DIR_CONFIG)) .map(String::trim).filter(s -> !s.isEmpty()).toList() ) .stream() From 193245d724fdc61d08734696a9edb358ed85f549 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 20 Aug 2025 23:00:16 +0800 Subject: [PATCH 44/78] remove unused import --- core/src/main/scala/kafka/server/KafkaConfig.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 8ac4f5b333c5d..db5041347d700 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -48,7 +48,6 @@ import org.apache.kafka.server.config.AbstractKafkaConfig.getMap import org.apache.kafka.server.config.{AbstractKafkaConfig, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs -import org.apache.kafka.server.util.Csv import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import scala.jdk.CollectionConverters._ From 7a60356f50ec4dce39b82d5c980112ecde1fd84f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 20 Aug 2025 23:36:40 +0800 Subject: [PATCH 45/78] addressed by comments --- .../scala/unit/kafka/log/UnifiedLogTest.scala | 23 +++++++++++++++++++ .../storage/internals/log/UnifiedLog.java | 4 +++- 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index d30d5a1040ee5..c0af31eb00ac2 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3000,6 +3000,29 @@ class UnifiedLogTest { assertEquals(segments, log.numberOfSegments, "There should be 3 segments remaining") } + @Test + def shouldDeleteLocalLogSegmentsWhenPolicyIsEmptyWithSizeRetention(): Unit = { + def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) + val recordSize = createRecords.sizeInBytes + val logConfig = LogTestUtils.createLogConfig( + segmentBytes = recordSize * 2, + retentionBytes = recordSize * 5, + localRetentionBytes = recordSize * 5, + cleanupPolicy = "" + ) + val log = createLog(logDir, logConfig) + + for (_ <- 0 until 10) + log.appendAsLeader(createRecords, 0) + + val segmentsBefore = log.numberOfSegments + log.updateHighWatermark(log.logEndOffset) + val deleteOldSegments = log.deleteOldSegments() + + assertTrue(log.numberOfSegments < segmentsBefore, "Some segments should be deleted due to size retention") + assertTrue(deleteOldSegments > 0, "At least one segment should be deleted") + } + @Test def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index ca32e4f086a61..017725c722356 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1887,8 +1887,10 @@ public int deleteOldSegments() throws IOException { return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments() + deleteRetentionMsBreachedSegments(); - } else { + } else if (config().compact) { return deleteLogStartOffsetBreachedSegments(); + } else { + return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments(); } } From d50e5930d0aa684d01143af730de4524f419a275 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 21 Aug 2025 18:44:22 +0800 Subject: [PATCH 46/78] addressed by comments --- .../org/apache/kafka/network/SocketServerConfigs.java | 3 --- .../apache/kafka/server/config/AbstractKafkaConfig.java | 6 ++---- .../org/apache/kafka/network/SocketServerConfigsTest.java | 8 +++++--- 3 files changed, 7 insertions(+), 10 deletions(-) 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 0f136f2e9f236..3d2b06353592c 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -204,9 +204,6 @@ public static List listenerListToEndPoints( ) { List results = new ArrayList<>(); for (String entry : input) { - if (entry.isEmpty()) { - continue; - } Matcher matcher = URI_PARSE_REGEXP.matcher(entry); if (!matcher.matches()) { throw new KafkaException("Unable to parse " + entry + " to a broker endpoint"); diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 3e4e8ab5c52f8..326c529927b82 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -39,12 +39,12 @@ import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.stream.Collectors; -import java.util.stream.Stream; /** * During moving {@link kafka.server.KafkaConfig} out of core AbstractKafkaConfig will be the future KafkaConfig @@ -81,9 +81,7 @@ public AbstractKafkaConfig(ConfigDef definition, Map originals, Map logDirs() { return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)) - .orElse(Stream.of(getString(ServerLogConfigs.LOG_DIR_CONFIG)) - .map(String::trim).filter(s -> !s.isEmpty()).toList() - ) + .orElse(Arrays.stream(getString(ServerLogConfigs.LOG_DIR_CONFIG).split(",")).toList()) .stream() .map(String::trim) .toList(); diff --git a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java index 7b65759a67e36..6b0ca02e7ccf8 100644 --- a/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java +++ b/server/src/test/java/org/apache/kafka/network/SocketServerConfigsTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.network; import org.apache.kafka.common.Endpoint; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; @@ -26,6 +27,7 @@ import java.util.Map; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class SocketServerConfigsTest { @Test @@ -48,9 +50,9 @@ public void testListenerListToEndPointsWithEmptyString() { @Test public void testListenerListToEndPointsWithBlankString() { - assertEquals(List.of(), - SocketServerConfigs.listenerListToEndPoints(List.of(" "), - SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); + KafkaException exception = assertThrows(KafkaException.class, () -> + SocketServerConfigs.listenerListToEndPoints(List.of(" "), SocketServerConfigs.DEFAULT_NAME_TO_SECURITY_PROTO)); + assertEquals("Unable to parse to a broker endpoint", exception.getMessage()); } @Test From ececcb767203294b747fe00bb84cbda7635534dc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 21 Aug 2025 19:18:52 +0800 Subject: [PATCH 47/78] updated the HeaderFrom headers --- .../java/org/apache/kafka/connect/transforms/HeaderFrom.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java index a4b40a38bcf43..262e2d89539da 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -67,7 +67,7 @@ public abstract class HeaderFrom> implements Transfor "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, NO_DEFAULT_VALUE, - new NonEmptyListValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Header names, in the same order as the field names listed in the fields configuration property.") .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, From 236092cf14d3d339ea231ca97a8248981d9c0175 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 21 Aug 2025 19:30:34 +0800 Subject: [PATCH 48/78] updated the controller.listener.names --- .../src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 7 +++++-- .../apache/kafka/server/config/AbstractKafkaConfig.java | 2 +- .../java/org/apache/kafka/server/config/KRaftConfigs.java | 2 +- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 46350bdf82ab6..e401fd002b9a0 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -298,7 +298,8 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") - assertBadConfigContainingMessage(props, "The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") + assertBadConfigContainingMessage(props, + "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") KafkaConfig.fromProps(props) @@ -318,7 +319,8 @@ class KafkaConfigTest { props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) - assertBadConfigContainingMessage(props, "controller.listener.names must contain at least one value when running KRaft with just the broker role") + assertBadConfigContainingMessage(props, + "Missing required configuration \"controller.listener.names\" which has no default value.") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") KafkaConfig.fromProps(props) @@ -1585,6 +1587,7 @@ class KafkaConfigTest { props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(ServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") + props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertEquals("You must set `node.id` to the same value as `broker.id`.", assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) } diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 326c529927b82..f981f592dfe92 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -112,7 +112,7 @@ public int requestTimeoutMs() { } public List controllerListenerNames() { - return Csv.parseCsvList(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)); + return getList(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG); } public ListenerName interBrokerListenerName() { 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 9dc886fb2ef2d..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 @@ -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, STRING, null, null, HIGH, CONTROLLER_LISTENER_NAMES_DOC) + .define(CONTROLLER_LISTENER_NAMES_CONFIG, LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), HIGH, CONTROLLER_LISTENER_NAMES_DOC) .define(SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SASL_MECHANISM_CONTROLLER_PROTOCOL_DOC) .defineInternal(CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS, LONG, CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS_DEFAULT, atLeast(100), MEDIUM, CONTROLLER_PERFORMANCE_SAMPLE_PERIOD_MS_DOC) .defineInternal(CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS, LONG, CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DEFAULT, atLeast(0), MEDIUM, CONTROLLER_PERFORMANCE_ALWAYS_LOG_THRESHOLD_MS_DOC) From 6824321ea43d1b67876429c9bd3e590faa0693df Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 22 Aug 2025 21:18:20 +0800 Subject: [PATCH 49/78] update the LogConfig validate remote storage logic --- core/src/test/scala/unit/kafka/log/LogConfigTest.scala | 2 +- .../org/apache/kafka/storage/internals/log/LogConfig.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index cf358802d978f..e23e16fa40a5f 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -298,7 +298,7 @@ class LogConfigTest { logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "delete") validateCleanupPolicy() logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, "") - assertThrows(classOf[ConfigException], () => validateCleanupPolicy()) + validateCleanupPolicy() } @ParameterizedTest(name = "testEnableRemoteLogStorage with sysRemoteStorageEnabled: {0}") diff --git a/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 9a83ae1d1b64f..793248b8952b6 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,8 +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.size() != 1 || !TopicConfig.CLEANUP_POLICY_DELETE.equals(cleanupPolicy.get(0))) { - throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete"); + if (!cleanupPolicy.isEmpty() && (cleanupPolicy.size() != 1 || !TopicConfig.CLEANUP_POLICY_DELETE.equals(cleanupPolicy.get(0)))) { + throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete or cleanup.policy is empty list."); } } From 5b77cb4645cbce9a59afcdc4c02071ecd64467c1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 22 Aug 2025 21:18:56 +0800 Subject: [PATCH 50/78] fix empty listener name test and update the advertised.listeners --- .../integration/kafka/server/QuorumTestHarness.scala | 10 ++++++---- .../org/apache/kafka/network/SocketServerConfigs.java | 2 +- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 3d5837b92d0d7..31420d7dae3b5 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -260,11 +260,13 @@ abstract class QuorumTestHarness extends Logging { props.setProperty(MetadataLogConfig.METADATA_LOG_DIR_CONFIG, metadataDir.getAbsolutePath) val proto = controllerListenerSecurityProtocol.toString val securityProtocolMaps = extraControllerSecurityProtocols().map(sc => sc + ":" + sc).mkString(",") - val listeners = extraControllerSecurityProtocols().map(sc => sc + "://localhost:0").mkString(",") - val listenerNames = extraControllerSecurityProtocols().mkString(",") + val listeners = extraControllerSecurityProtocols().map(sc => sc + "://localhost:0").mkString(",").trim + val listenerNames = extraControllerSecurityProtocols().mkString(",").trim props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto,$securityProtocolMaps") - props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0,$listeners") - props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, s"CONTROLLER,$listenerNames") + props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, + if (listeners.isEmpty) "CONTROLLER://localhost:0" else s"CONTROLLER://localhost:0,$listeners") + props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, + if (listeners.isEmpty) "CONTROLLER" else s"CONTROLLER,$listenerNames") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0") props.setProperty(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") val config = new KafkaConfig(props) diff --git a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java index 3d2b06353592c..7f123714fc151 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -156,7 +156,7 @@ public class SocketServerConfigs { 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, HIGH, ADVERTISED_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) From 525cea6dc31eaf1a760813f4ce9c19896952961a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 22 Aug 2025 21:46:28 +0800 Subject: [PATCH 51/78] update the UnifiedLog logic and new test --- .../scala/unit/kafka/log/UnifiedLogTest.scala | 34 ++++++++++++++++--- .../storage/internals/log/UnifiedLog.java | 11 ++++-- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index c0af31eb00ac2..ba8c995345e9c 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3006,23 +3006,49 @@ class UnifiedLogTest { val recordSize = createRecords.sizeInBytes val logConfig = LogTestUtils.createLogConfig( segmentBytes = recordSize * 2, - retentionBytes = recordSize * 5, - localRetentionBytes = recordSize * 5, - cleanupPolicy = "" + localRetentionBytes = recordSize / 2, + cleanupPolicy = "", + remoteLogStorageEnable = true ) - val log = createLog(logDir, logConfig) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) for (_ <- 0 until 10) log.appendAsLeader(createRecords, 0) val segmentsBefore = log.numberOfSegments log.updateHighWatermark(log.logEndOffset) + log.updateHighestOffsetInRemoteStorage(log.logEndOffset - 1) val deleteOldSegments = log.deleteOldSegments() assertTrue(log.numberOfSegments < segmentsBefore, "Some segments should be deleted due to size retention") assertTrue(deleteOldSegments > 0, "At least one segment should be deleted") } + @Test + def shouldDeleteLocalLogSegmentsWhenPolicyIsEmptyWithMsRetention(): Unit = { + def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) + val recordSize = createRecords.sizeInBytes + val logConfig = LogTestUtils.createLogConfig( + segmentBytes = recordSize * 2, + localRetentionMs = 10000, + cleanupPolicy = "", + remoteLogStorageEnable = true + ) + val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) + + for (_ <- 0 until 10) + log.appendAsLeader(createRecords, 0) + + // mark the oldest segment as older the retention.ms + log.logSegments.asScala.head.setLastModified(mockTime.milliseconds - 20000) + + val segments = log.numberOfSegments + log.updateHighWatermark(log.logEndOffset) + log.updateHighestOffsetInRemoteStorage(log.logEndOffset - 1) + log.deleteOldSegments() + assertTrue(log.numberOfSegments < segments, "Old segments should be deleted") + } + @Test def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete(): Unit = { def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes, timestamp = 10L) diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index 017725c722356..d87d7c0ed1134 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1879,7 +1879,8 @@ private int deleteSegments(List deletable, SegmentDeletionReason rea /** * If topic deletion is enabled, delete any local log segments that have either expired due to time based - * retention or because the log size is > retentionSize. Whether or not deletion is enabled, delete any local + * retention or because the log size is > retentionSize. Empty cleanup.policy with remote storage enabled + * behaves the same as deletion policy. Whether or not deletion is enabled, delete any local * log segments that are before the log start offset */ public int deleteOldSegments() throws IOException { @@ -1890,7 +1891,13 @@ public int deleteOldSegments() throws IOException { } else if (config().compact) { return deleteLogStartOffsetBreachedSegments(); } else { - return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments(); + if (remoteLogEnabledAndRemoteCopyEnabled()) { + return deleteLogStartOffsetBreachedSegments() + + deleteRetentionSizeBreachedSegments() + + deleteRetentionMsBreachedSegments(); + } else { + return deleteLogStartOffsetBreachedSegments(); + } } } From 9e419d638a3bad84205a052b611586b131e6200e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 22 Aug 2025 22:01:40 +0800 Subject: [PATCH 52/78] fix build error --- .../java/org/apache/kafka/network/SocketServerConfigs.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 7f123714fc151..19eea1fd0e2aa 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -154,9 +154,9 @@ public class SocketServerConfigs { public static final int NUM_NETWORK_THREADS_DEFAULT = 3; public static final String NUM_NETWORK_THREADS_DOC = "The number of threads that the server uses for receiving requests from the network and sending responses to the network. Noted: each listener (except for controller listener) creates its own thread pool."; - public static final ConfigDef CONFIG_DEF = new ConfigDef() + 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(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) From 92f1d8fd37d71981de63a3a13b2e4515b3e6a259 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 23 Aug 2025 12:02:33 +0800 Subject: [PATCH 53/78] addressed by comment --- .../java/org/apache/kafka/common/config/TopicConfig.java | 5 +++-- .../scala/integration/kafka/server/QuorumTestHarness.scala | 4 ++-- .../org/apache/kafka/server/config/ServerLogConfigs.java | 6 +++++- .../org/apache/kafka/storage/internals/log/LogConfig.java | 2 +- .../org/apache/kafka/storage/internals/log/UnifiedLog.java | 6 +++--- 5 files changed, 14 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java index c9a284e3578bf..e97c39bc61911 100755 --- a/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/TopicConfig.java @@ -159,9 +159,10 @@ public class TopicConfig { "log compaction, which retains the latest value for each key. " + "It is also possible to specify both policies in a comma-separated list (e.g. \"delete,compact\"). " + "In this case, old segments will be discarded per the retention time and size configuration, " + - "while retained segments will be compacted." + + "while retained segments will be compacted. " + "An empty list means infinite retention - no cleanup policies will be applied and log segments " + - "will be retained indefinitely."; + "will be retained indefinitely. Note that with remote storage enabled, local retention limits " + + "(log.local.retention.ms and log.local.retention.bytes) are still applied to local segments."; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG = "unclean.leader.election.enable"; public static final String UNCLEAN_LEADER_ELECTION_ENABLE_DOC = "Indicates whether to enable replicas " + diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 31420d7dae3b5..6af0932690f92 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -260,8 +260,8 @@ abstract class QuorumTestHarness extends Logging { props.setProperty(MetadataLogConfig.METADATA_LOG_DIR_CONFIG, metadataDir.getAbsolutePath) val proto = controllerListenerSecurityProtocol.toString val securityProtocolMaps = extraControllerSecurityProtocols().map(sc => sc + ":" + sc).mkString(",") - val listeners = extraControllerSecurityProtocols().map(sc => sc + "://localhost:0").mkString(",").trim - val listenerNames = extraControllerSecurityProtocols().mkString(",").trim + val listeners = extraControllerSecurityProtocols().map(sc => sc + "://localhost:0").mkString(",") + val listenerNames = extraControllerSecurityProtocols().mkString(",") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto,$securityProtocolMaps") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, if (listeners.isEmpty) "CONTROLLER://localhost:0" else s"CONTROLLER://localhost:0,$listeners") diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 2c2d44ef012e5..69d078258e9d6 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -71,7 +71,11 @@ public class ServerLogConfigs { public static final String LOG_CLEANUP_POLICY_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG); public static final String LOG_CLEANUP_POLICY_DEFAULT = TopicConfig.CLEANUP_POLICY_DELETE; - public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. A comma separated list of valid policies."; + public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. " + + "A comma separated list of valid policies. The \"delete\" policy will discard old segments when their " + + "retention time or size limit has been reached. The \"compact\" policy will enable log compaction. " + + "An empty list means infinite retention - no cleanup policies will be applied by default. " + + "When remote storage is enabled, local retention policies are still enforced to manage local storage usage."; public static final String LOG_INDEX_SIZE_MAX_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); public static final int LOG_INDEX_SIZE_MAX_BYTES_DEFAULT = 10 * 1024 * 1024; diff --git a/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 793248b8952b6..c156ba1f14c0c 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 @@ -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); 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 is empty list."); + throw new ConfigException("Remote log storage only supports topics with cleanup.policy=delete or cleanup.policy being an empty list."); } } diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index d87d7c0ed1134..57d128647d4f4 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1879,9 +1879,9 @@ private int deleteSegments(List deletable, SegmentDeletionReason rea /** * If topic deletion is enabled, delete any local log segments that have either expired due to time based - * retention or because the log size is > retentionSize. Empty cleanup.policy with remote storage enabled - * behaves the same as deletion policy. Whether or not deletion is enabled, delete any local - * log segments that are before the log start offset + * retention or because the log size is > retentionSize. Empty cleanup.policy is the same as delete with + * infinite retention, so we only need to delete local segments if remote storage is enabled. Whether or + * not deletion is enabled, delete any local log segments that are before the log start offset */ public int deleteOldSegments() throws IOException { if (config().delete) { From 3c36c66a1770116a5cdf93dcf9fce9e916e42df1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 23 Aug 2025 13:02:58 +0800 Subject: [PATCH 54/78] updated the HeaderFrom --- .../java/org/apache/kafka/connect/transforms/HeaderFrom.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java index 262e2d89539da..a4b40a38bcf43 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -67,7 +67,7 @@ public abstract class HeaderFrom> implements Transfor "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, NO_DEFAULT_VALUE, - ConfigDef.ValidList.anyNonDuplicateValues(false, false), + new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Header names, in the same order as the field names listed in the fields configuration property.") .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, From be799470b377eaba71718d29228d2dc4178c7b01 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 23 Aug 2025 13:03:06 +0800 Subject: [PATCH 55/78] Fix other tests --- .../scala/unit/kafka/KafkaConfigTest.scala | 39 +++++++++++++++---- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- .../AddPartitionsToTxnManagerTest.java | 5 ++- 3 files changed, 36 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 81f2c8e5e08f2..25fcff11dd9fa 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -57,13 +57,13 @@ class KafkaConfigTest { properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) assertBadConfigContainingMessage(properties, - "If using process.roles, either controller.quorum.bootstrap.servers must contain the set of bootstrap controllers or controller.quorum.voters must contain a parseable set of controllers.") + "Missing required configuration \"controller.listener.names\" which has no default value.") - properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertBadConfigContainingMessage(properties, - "requirement failed: controller.listener.names must contain at least one value when running KRaft with just the broker role") + "If using process.roles, either controller.quorum.bootstrap.servers must contain the set of bootstrap controllers or controller.quorum.voters must contain a parseable set of controllers.") - properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") + properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") KafkaConfig.fromProps(properties) } @@ -82,6 +82,10 @@ class KafkaConfigTest { "Invalid value -1 for configuration node.id: Value must be at least 0") properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + assertBadConfigContainingMessage(properties, + "Missing required configuration \"controller.listener.names\" which has no default value.") + + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") assertBadConfigContainingMessage(properties, "If using process.roles, either controller.quorum.bootstrap.servers must contain the set of bootstrap controllers or controller.quorum.voters must contain a parseable set of controllers.") @@ -90,15 +94,34 @@ class KafkaConfigTest { "requirement failed: The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") - assertBadConfigContainingMessage(properties, - "No security protocol defined for listener CONTROLLER") + properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT") + KafkaConfig.fromProps(properties) + } + @Test + def testControllerListenerNamesMismatch(): Unit = { + val properties = new Properties() + properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") + properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "OTHER") + properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT") + assertBadConfigContainingMessage(properties, "requirement failed: The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") + } - properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") - KafkaConfig.fromProps(properties) + @Test + def testControllerSecurityProtocolMissing(): Unit = { + val properties = new Properties() + properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") + properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) + properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "OTHER") + properties.put(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG, "localhost:9092") + properties.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:9092") + + assertBadConfigContainingMessage(properties, "No security protocol defined for listener CONTROLLER") } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 79ad3ad924e84..0d54711aa892f 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -220,7 +220,7 @@ class KafkaConfigTest { // but not duplicate names props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "HOST://localhost:9091,HOST://localhost:9091") - assertBadConfigContainingMessage(props, "Each listener must have a different name") + assertBadConfigContainingMessage(props, "Configuration 'advertised.listeners' values must not be duplicated.") } @Test diff --git a/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java b/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java index 985d183426421..90a8132e8eabb 100644 --- a/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java +++ b/server/src/test/java/org/apache/kafka/server/transaction/AddPartitionsToTxnManagerTest.java @@ -85,7 +85,10 @@ public class AddPartitionsToTxnManagerTest { private final MockTime time = new MockTime(); private final AbstractKafkaConfig config = new AbstractKafkaConfig( AbstractKafkaConfig.CONFIG_DEF, - Map.of(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker", KRaftConfigs.NODE_ID_CONFIG, "1"), + Map.of( + KRaftConfigs.PROCESS_ROLES_CONFIG, "broker", + KRaftConfigs.NODE_ID_CONFIG, "1", + KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER"), Map.of(), false) { }; private final AddPartitionsToTxnManager addPartitionsToTxnManager = From 1b4aa7fe06c4739b634168cd5da49edfe8bdb9fb Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 27 Aug 2025 19:24:33 +0800 Subject: [PATCH 56/78] addressed by comments --- .../scala/unit/kafka/server/KafkaConfigTest.scala | 4 ++-- .../apache/kafka/server/config/ServerLogConfigs.java | 12 +++++------- .../kafka/server/config/AbstractKafkaConfig.java | 3 +-- .../kafka/storage/internals/log/LogConfig.java | 2 +- 4 files changed, 9 insertions(+), 12 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 0d54711aa892f..dd4995c0f540a 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -826,8 +826,8 @@ class KafkaConfigTest { case SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") case ServerLogConfigs.NUM_PARTITIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case ServerLogConfigs.LOG_DIRS_CONFIG => // ignore list - case ServerLogConfigs.LOG_DIR_CONFIG => // ignore string + case ServerLogConfigs.LOG_DIRS_CONFIG => assertPropertyInvalid(baseProperties, name, "") + case ServerLogConfigs.LOG_DIR_CONFIG => assertPropertyInvalid(baseProperties, name, "") case ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", Records.LOG_OVERHEAD - 1) case ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 69d078258e9d6..57fcb0a88b671 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -21,6 +21,8 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.server.record.BrokerCompressionType; +import java.util.List; + import static org.apache.kafka.server.config.ServerTopicConfigSynonyms.LOG_PREFIX; /** @@ -36,8 +38,8 @@ public class ServerLogConfigs { public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs"; public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir"; - public static final String LOG_DIR_DEFAULT = "/tmp/kafka-logs"; - public static final String LOG_DIR_DOC = "The directory in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)"; + public static final List LOG_DIR_DEFAULT = List.of("/tmp/kafka-logs"); + public static final String LOG_DIR_DOC = "The directories in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)"; public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used."; public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); @@ -71,11 +73,7 @@ public class ServerLogConfigs { public static final String LOG_CLEANUP_POLICY_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.CLEANUP_POLICY_CONFIG); public static final String LOG_CLEANUP_POLICY_DEFAULT = TopicConfig.CLEANUP_POLICY_DELETE; - public static final String LOG_CLEANUP_POLICY_DOC = "The default cleanup policy for segments beyond the retention window. " + - "A comma separated list of valid policies. The \"delete\" policy will discard old segments when their " + - "retention time or size limit has been reached. The \"compact\" policy will enable log compaction. " + - "An empty list means infinite retention - no cleanup policies will be applied by default. " + - "When remote storage is enabled, local retention policies are still enforced to manage local storage usage."; + public static final String LOG_CLEANUP_POLICY_DOC = TopicConfig.CLEANUP_POLICY_DOC; public static final String LOG_INDEX_SIZE_MAX_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG); public static final int LOG_INDEX_SIZE_MAX_BYTES_DEFAULT = 10 * 1024 * 1024; diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index f981f592dfe92..7c0306e97a7c9 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -39,7 +39,6 @@ import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.LogConfig; -import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -81,7 +80,7 @@ public AbstractKafkaConfig(ConfigDef definition, Map originals, Map logDirs() { return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)) - .orElse(Arrays.stream(getString(ServerLogConfigs.LOG_DIR_CONFIG).split(",")).toList()) + .orElse(getList(ServerLogConfigs.LOG_DIR_CONFIG)) .stream() .map(String::trim) .toList(); 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 c156ba1f14c0c..e5a19103b6634 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,7 +144,7 @@ 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, STRING, ServerLogConfigs.LOG_DIR_DEFAULT, HIGH, ServerLogConfigs.LOG_DIR_DOC) + .define(ServerLogConfigs.LOG_DIR_CONFIG, LIST, ServerLogConfigs.LOG_DIR_DEFAULT, ConfigDef.ValidList.anyNonDuplicateValues(false, true), 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) From 234fc0b957382b8b8a5e30dcd7677c7f5a938213 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 27 Aug 2025 20:37:40 +0800 Subject: [PATCH 57/78] update the AbstractKafkaConfig --- .../org/apache/kafka/server/config/AbstractKafkaConfig.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 7c0306e97a7c9..97a7e84e3254b 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -79,11 +79,7 @@ public AbstractKafkaConfig(ConfigDef definition, Map originals, Map logDirs() { - return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)) - .orElse(getList(ServerLogConfigs.LOG_DIR_CONFIG)) - .stream() - .map(String::trim) - .toList(); + return Optional.ofNullable(getList(ServerLogConfigs.LOG_DIRS_CONFIG)).orElse(getList(ServerLogConfigs.LOG_DIR_CONFIG)); } public int numIoThreads() { From a50002db2c98dc625d0c10f224640972bf7ccae2 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 28 Aug 2025 18:37:30 +0800 Subject: [PATCH 58/78] addressed by comments --- .../clients/consumer/ConsumerConfig.java | 3 +-- .../clients/producer/ProducerConfig.java | 3 +-- .../kafka/connect/transforms/HeaderFrom.java | 6 ++--- .../scala/unit/kafka/KafkaConfigTest.scala | 2 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 23 +++++++++++-------- docs/configuration.html | 1 + .../kafka/server/config/ServerLogConfigs.java | 2 +- .../server/config/AbstractKafkaConfig.java | 1 - .../storage/internals/log/LogConfig.java | 2 +- 9 files changed, 22 insertions(+), 21 deletions(-) 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 d72eaa495e793..ee62dc9561b41 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -37,7 +37,6 @@ import org.apache.kafka.common.utils.Utils; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; @@ -614,7 +613,7 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC) .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, - Collections.emptyList(), + List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_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 09240d31ccf00..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 @@ -35,7 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -507,7 +506,7 @@ public class ProducerConfig extends AbstractConfig { Importance.MEDIUM, PARTITIONER_CLASS_DOC) .define(INTERCEPTOR_CLASSES_CONFIG, Type.LIST, - Collections.emptyList(), + List.of(), ConfigDef.ValidList.anyNonDuplicateValues(true, false), Importance.LOW, INTERCEPTOR_CLASSES_DOC) diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java index a4b40a38bcf43..8b9030066ae09 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HeaderFrom.java @@ -61,13 +61,11 @@ public abstract class HeaderFrom> implements Transfor public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELDS_FIELD, ConfigDef.Type.LIST, - NO_DEFAULT_VALUE, - new NonEmptyListValidator(), + NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Field names in the record whose values are to be copied or moved to headers.") .define(HEADERS_FIELD, ConfigDef.Type.LIST, - NO_DEFAULT_VALUE, - new NonEmptyListValidator(), + NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Header names, in the same order as the field names listed in the fields configuration property.") .define(OPERATION_FIELD, ConfigDef.Type.STRING, NO_DEFAULT_VALUE, diff --git a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala index 25fcff11dd9fa..9ca8b42cd14be 100644 --- a/core/src/test/scala/unit/kafka/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/KafkaConfigTest.scala @@ -113,7 +113,7 @@ class KafkaConfigTest { } @Test - def testControllerSecurityProtocolMissing(): Unit = { + def testControllerSecurityProtocolMapMissing(): Unit = { val properties = new Properties() properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") properties.put(KRaftConfigs.NODE_ID_CONFIG, 0) diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index f48aaeee914fc..c4cb7a9f7b413 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -3213,27 +3213,32 @@ class UnifiedLogTest { @Test def shouldDeleteLocalLogSegmentsWhenPolicyIsEmptyWithMsRetention(): Unit = { - def createRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L) - val recordSize = createRecords.sizeInBytes + val oldTimestamp = mockTime.milliseconds - 20000 + def oldRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = oldTimestamp) + val recordSize = oldRecords.sizeInBytes val logConfig = LogTestUtils.createLogConfig( segmentBytes = recordSize * 2, - localRetentionMs = 10000, + localRetentionMs = 5000, cleanupPolicy = "", remoteLogStorageEnable = true ) val log = createLog(logDir, logConfig, remoteStorageSystemEnable = true) for (_ <- 0 until 10) - log.appendAsLeader(createRecords, 0) + log.appendAsLeader(oldRecords, 0) - // mark the oldest segment as older the retention.ms - log.logSegments.asScala.head.setLastModified(mockTime.milliseconds - 20000) + def newRecords = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = mockTime.milliseconds) + for (_ <- 0 until 5) + log.appendAsLeader(newRecords, 0) + + val segmentsBefore = log.numberOfSegments - val segments = log.numberOfSegments log.updateHighWatermark(log.logEndOffset) log.updateHighestOffsetInRemoteStorage(log.logEndOffset - 1) - log.deleteOldSegments() - assertTrue(log.numberOfSegments < segments, "Old segments should be deleted") + val deleteOldSegments = log.deleteOldSegments() + + assertTrue(log.numberOfSegments < segmentsBefore, "Some segments should be deleted due to time retention") + assertTrue(deleteOldSegments > 0, "At least one segment should be deleted") } @Test diff --git a/docs/configuration.html b/docs/configuration.html index 425d306a4dc12..0c114f79ef4c9 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -26,6 +26,7 @@

log.dirs
  • process.roles
  • controller.quorum.bootstrap.servers +
  • controller.listener.names Topic configurations and defaults are discussed in more detail below. diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 57fcb0a88b671..4457b142d5a2f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -39,7 +39,7 @@ public class ServerLogConfigs { public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs"; public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir"; public static final List LOG_DIR_DEFAULT = List.of("/tmp/kafka-logs"); - public static final String LOG_DIR_DOC = "The directories in which the log data is kept (supplemental for " + LOG_DIRS_CONFIG + " property)"; + public static final String LOG_DIR_DOC = "A comma-separated list of the directories where the log data is stored. (supplemental to " + LOG_DIRS_CONFIG + " property)"; public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used."; public static final String LOG_SEGMENT_BYTES_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.SEGMENT_BYTES_CONFIG); diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java index 97a7e84e3254b..7be9fcf23f503 100644 --- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java +++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java @@ -138,7 +138,6 @@ public Map effectiveListenerSecurityProtocolMap( // are not included in 'listeners' config when process.roles=broker) if (controllerListenerNames().stream().anyMatch(AbstractKafkaConfig::isSslOrSasl) || getList(SocketServerConfigs.LISTENERS_CONFIG).stream() - .map(String::trim) .anyMatch(listenerName -> isSslOrSasl(parseListenerName(listenerName)))) { return mapValue; } else { 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 e5a19103b6634..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,7 +144,7 @@ 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, true), HIGH, ServerLogConfigs.LOG_DIR_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) From 61c1b1039d2a1ad01a2d7c75dfc2a89f590b4917 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 28 Aug 2025 19:10:50 +0800 Subject: [PATCH 59/78] fix the consumer and producer test --- .../clients/consumer/KafkaConsumerTest.java | 24 +++++++++++++++++++ .../clients/producer/KafkaProducerTest.java | 23 ++++++++++++++++++ .../kafka/test/MockConsumerInterceptor.java | 1 + .../kafka/test/MockProducerInterceptor.java | 1 + 4 files changed, 49 insertions(+) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 82d080d682078..927b02000ba69 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -713,6 +713,30 @@ public void testInterceptorConstructorClose(GroupProtocol groupProtocol) { } } + @ParameterizedTest + @EnumSource(GroupProtocol.class) + public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) { + final int targetInterceptor = 1; + + try { + Properties props = new Properties(); + props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); + props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); + + MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); + + assertThrows(KafkaException.class, () -> newConsumer( + props, new StringDeserializer(), new StringDeserializer())); + + assertEquals(1, MockConsumerInterceptor.CONFIG_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); + + } finally { + MockConsumerInterceptor.resetCounters(); + } + } + @ParameterizedTest @EnumSource(GroupProtocol.class) public void testPause(GroupProtocol groupProtocol) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index a571f06599e16..a8525403ca8de 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -638,6 +638,29 @@ props, new StringSerializer(), new StringSerializer())) { } } + @Test + public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances() { + final int targetInterceptor = 1; + try { + Properties props = new Properties(); + props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); + props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, org.apache.kafka.test.MockProducerInterceptor.class.getName()); + props.setProperty(MockProducerInterceptor.APPEND_STRING_PROP, "something"); + + MockProducerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); + + assertThrows(KafkaException.class, () -> + new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()) + ); + + assertEquals(1, MockProducerInterceptor.CONFIG_COUNT.get()); + assertEquals(1, MockProducerInterceptor.CLOSE_COUNT.get()); + + } finally { + MockProducerInterceptor.resetCounters(); + } + } + @Test public void testPartitionerClose() { try { diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java index a1d676e15a3aa..715486c1ae36b 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -118,6 +118,7 @@ public static void resetCounters() { CONFIG_COUNT.set(0); THROW_CONFIG_EXCEPTION.set(0); CLUSTER_META.set(null); + THROW_ON_CONFIG_EXCEPTION_THRESHOLD.set(0); CLUSTER_ID_BEFORE_ON_CONSUME.set(NO_CLUSTER_ID); } diff --git a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java index acc69ab44e31a..9e69f57c96f2c 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java @@ -110,6 +110,7 @@ public static void resetCounters() { ON_SUCCESS_COUNT.set(0); ON_ERROR_COUNT.set(0); ON_ERROR_WITH_METADATA_COUNT.set(0); + THROW_ON_CONFIG_EXCEPTION_THRESHOLD.set(0); CLUSTER_META.set(null); CLUSTER_ID_BEFORE_ON_ACKNOWLEDGEMENT.set(NO_CLUSTER_ID); } From 7ac09fdefb642bc7e62cf742ba953e2f4fc26a89 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 28 Aug 2025 21:05:46 +0800 Subject: [PATCH 60/78] update the ValueToKey --- .../java/org/apache/kafka/connect/transforms/ValueToKey.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 24cdec2249ab1..dd81898d61a82 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 @@ -46,7 +46,7 @@ public class ValueToKey> implements Transformation public static final String REPLACE_NULL_WITH_DEFAULT_CONFIG = "replace.null.with.default"; public static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.ValidList.anyNonDuplicateValues(false, false), ConfigDef.Importance.HIGH, "Field names on the record value to extract as the record key.") .define(REPLACE_NULL_WITH_DEFAULT_CONFIG, ConfigDef.Type.BOOLEAN, true, ConfigDef.Importance.MEDIUM, "Whether to replace fields that have a default value and that are null to the default value. When set to true, the default value is used, otherwise null is used."); From 5482d6462474f80569e3c02a7857500d2b880a63 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 28 Aug 2025 21:08:17 +0800 Subject: [PATCH 61/78] spotlessApply --- .../java/org/apache/kafka/connect/transforms/ValueToKey.java | 1 - 1 file changed, 1 deletion(-) 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 dd81898d61a82..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 @@ -28,7 +28,6 @@ import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; import org.apache.kafka.connect.transforms.util.SimpleConfig; import java.util.HashMap; From 8d2d3779ea13c80a73a01797f74e845c37cc177e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 29 Aug 2025 18:52:44 +0800 Subject: [PATCH 62/78] updated the test naming --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 2 +- .../org/apache/kafka/clients/producer/KafkaProducerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 927b02000ba69..228f272528409 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -715,7 +715,7 @@ public void testInterceptorConstructorClose(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) { + public void testInterceptorConstructorExceptionShouldCloseCreatedInstances(GroupProtocol groupProtocol) { final int targetInterceptor = 1; try { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index a8525403ca8de..51d2cc27a43a6 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -639,7 +639,7 @@ props, new StringSerializer(), new StringSerializer())) { } @Test - public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances() { + public void testInterceptorConstructorExceptionShouldCloseCreatedInstances() { final int targetInterceptor = 1; try { Properties props = new Properties(); From 8f806e4fe5b86011da50cefcac55e9b7bdee480d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 29 Aug 2025 18:55:26 +0800 Subject: [PATCH 63/78] updated the test naming --- .../org/apache/kafka/clients/consumer/KafkaConsumerTest.java | 2 +- .../org/apache/kafka/clients/producer/KafkaProducerTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 228f272528409..0718032667759 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -715,7 +715,7 @@ public void testInterceptorConstructorClose(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testInterceptorConstructorExceptionShouldCloseCreatedInstances(GroupProtocol groupProtocol) { + public void testInterceptorConstructorExceptionShouldCloseCreatedInstance(GroupProtocol groupProtocol) { final int targetInterceptor = 1; try { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 51d2cc27a43a6..3d958a7e0b32d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -639,7 +639,7 @@ props, new StringSerializer(), new StringSerializer())) { } @Test - public void testInterceptorConstructorExceptionShouldCloseCreatedInstances() { + public void testInterceptorConstructorExceptionShouldCloseCreatedInstance() { final int targetInterceptor = 1; try { Properties props = new Properties(); From d2b7bdad65e2d2c34c37a5338aff6da3b8b7e548 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 30 Aug 2025 11:11:27 +0800 Subject: [PATCH 64/78] addressed by comments --- .../apache/kafka/common/config/ConfigDef.java | 2 +- .../security/ssl/DefaultSslEngineFactory.java | 4 +-- .../runtime/rest/RestServerConfig.java | 33 +------------------ .../main/scala/kafka/utils/CoreUtils.scala | 2 +- .../kafka/network/SocketServerConfigs.java | 2 +- .../storage/internals/log/UnifiedLog.java | 4 +++ 6 files changed, 10 insertions(+), 37 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 cb394f6545c0e..466a187c52afc 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 @@ -1025,7 +1025,7 @@ public static ValidList in(String... validStrings) { public static ValidList in(boolean isEmptyAllowed, String... validStrings) { if (!isEmptyAllowed && validStrings.length == 0) { - throw new IllegalArgumentException("Valid strings list cannot be empty for inNonEmpty validator"); + throw new IllegalArgumentException("At least one valid string must be provided when empty values are not allowed"); } return new ValidList(List.of(validStrings), isEmptyAllowed, false); } diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java index 0a3d587df9060..a5068dc83abce 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactory.java @@ -137,14 +137,14 @@ public void configure(Map configs) { SecurityUtils.addConfiguredSecurityProviders(this.configs); List cipherSuitesList = (List) configs.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - if (cipherSuitesList != null && !cipherSuitesList.isEmpty()) { + if (!cipherSuitesList.isEmpty()) { this.cipherSuites = cipherSuitesList.toArray(new String[0]); } else { this.cipherSuites = null; } List enabledProtocolsList = (List) configs.get(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); - if (enabledProtocolsList != null && !enabledProtocolsList.isEmpty()) { + if (!enabledProtocolsList.isEmpty()) { this.enabledProtocols = enabledProtocolsList.toArray(new String[0]); } else { this.enabledProtocols = null; diff --git a/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 22623f74d6bcc..9e7d2f7ffb11d 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 @@ -147,7 +147,7 @@ public static void addPublicConfig(ConfigDef configDef) { .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - new AdminListenersValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, true), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } @@ -328,37 +328,6 @@ public String toString() { } } - private static class AdminListenersValidator implements ConfigDef.Validator { - @Override - public void ensureValid(String name, Object value) { - if (value == null) { - return; - } - - if (!(value instanceof List items)) { - throw new ConfigException("Invalid value type for admin.listeners (expected list)."); - } - - if (items.isEmpty()) { - return; - } - - for (Object item : items) { - if (!(item instanceof String)) { - throw new ConfigException("Invalid type for admin.listeners (expected String)."); - } - if (Utils.isBlank((String) item)) { - throw new ConfigException("Empty URL found when parsing admin.listeners list."); - } - } - } - - @Override - public String toString() { - return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443."; - } - } - private static class ResponseHttpHeadersValidator implements ConfigDef.Validator { @Override public void ensureValid(String name, Object value) { diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 759f8b4938b4f..15f5056bb2031 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -128,7 +128,7 @@ object CoreUtils { private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: java.util.List[String]): Unit = { val distinctPorts = endpoints.map(_.port).distinct - require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: ${listeners.stream().collect(Collectors.joining(","))}") + require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: $listeners") } def listenerListToEndPoints(listeners: java.util.List[String], securityProtocolMap: java.util.Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = { 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 19eea1fd0e2aa..5aa0ddcf9447b 100644 --- a/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java +++ b/server/src/main/java/org/apache/kafka/network/SocketServerConfigs.java @@ -63,7 +63,7 @@ public class SocketServerConfigs { "is assumed if no explicit mapping is provided and no other security protocol is in use."; public static final String LISTENERS_CONFIG = "listeners"; - public static final List LISTENERS_DEFAULT = List.of("PLAINTEXT://:9092"); + public static final String LISTENERS_DEFAULT = "PLAINTEXT://:9092"; public static final String LISTENERS_DOC = String.format("Listener List - Comma-separated list of URIs we will listen on and the listener names." + " If the listener name is not a security protocol, %s must also be set.%n" + " Listener names and port numbers must be unique unless one listener is an IPv4 address and the other listener is an IPv6 address (for the same port).%n" + diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index 555fbaf4d7405..a12e2b5963dc2 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1918,11 +1918,15 @@ public int deleteOldSegments() throws IOException { } else if (config().compact) { return deleteLogStartOffsetBreachedSegments(); } else { + // If cleanup.policy is empty and remote storage is enable, the local log segments will + // be cleaned based on the values of log.local.retention.bytes and log.local.retention.ms. if (remoteLogEnabledAndRemoteCopyEnabled()) { return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments() + deleteRetentionMsBreachedSegments(); } else { + // If cleanup.policy is empty and remote storage is disabled, we should not delete any local + // log segments. return deleteLogStartOffsetBreachedSegments(); } } From f2edfc3dca42332a0c1c453d7183a0efcac4b9fc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 30 Aug 2025 11:28:43 +0800 Subject: [PATCH 65/78] addressed by comments --- .../clients/consumer/KafkaConsumerTest.java | 36 ++++++++++++++- .../clients/producer/KafkaProducerTest.java | 45 +++++++++++++++++-- .../main/scala/kafka/utils/CoreUtils.scala | 1 - 3 files changed, 75 insertions(+), 7 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 0718032667759..60063e5226888 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -715,14 +715,15 @@ public void testInterceptorConstructorClose(GroupProtocol groupProtocol) { @ParameterizedTest @EnumSource(GroupProtocol.class) - public void testInterceptorConstructorExceptionShouldCloseCreatedInstance(GroupProtocol groupProtocol) { + public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances(GroupProtocol groupProtocol) { final int targetInterceptor = 1; try { Properties props = new Properties(); props.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name()); props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, MockConsumerInterceptor.class.getName()); + props.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, + CloseInterceptor.class.getName() + "," + MockConsumerInterceptor.class.getName()); MockConsumerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); @@ -732,8 +733,10 @@ public void testInterceptorConstructorExceptionShouldCloseCreatedInstance(GroupP assertEquals(1, MockConsumerInterceptor.CONFIG_COUNT.get()); assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, CloseInterceptor.CLOSE_COUNT.get()); } finally { MockConsumerInterceptor.resetCounters(); + CloseInterceptor.resetCounters(); } } @@ -3797,4 +3800,33 @@ public void withPluginMetrics(PluginMetrics metrics) { metrics.addMetric(name, (Measurable) (config, now) -> VALUE); } } + + public static class CloseInterceptor implements ConsumerInterceptor { + + public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); + + @Override + public ConsumerRecords onConsume(ConsumerRecords records) { + return null; + } + + @Override + public void onCommit(Map offsets) { + // no-op + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs) { + // no-op + } + + public static void resetCounters() { + CLOSE_COUNT.set(0); + } + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 3d958a7e0b32d..bf27fbaa11f37 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -235,7 +235,7 @@ public void testOverwriteAcksAndRetriesForIdempotentProducers() { ProducerConfig config = new ProducerConfig(props); assertTrue(config.getBoolean(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG)); assertTrue(Stream.of("-1", "all").anyMatch(each -> each.equalsIgnoreCase(config.getString(ProducerConfig.ACKS_CONFIG)))); - assertEquals((int) config.getInt(ProducerConfig.RETRIES_CONFIG), Integer.MAX_VALUE); + assertEquals(Integer.MAX_VALUE, (int) config.getInt(ProducerConfig.RETRIES_CONFIG)); assertTrue(config.getString(ProducerConfig.CLIENT_ID_CONFIG).equalsIgnoreCase("producer-" + config.getString(ProducerConfig.TRANSACTIONAL_ID_CONFIG))); } @@ -639,25 +639,28 @@ props, new StringSerializer(), new StringSerializer())) { } @Test - public void testInterceptorConstructorExceptionShouldCloseCreatedInstance() { + public void testInterceptorConstructorConfigurationWithExceptionShouldCloseRemainingInstances() { final int targetInterceptor = 1; try { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999"); - props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, org.apache.kafka.test.MockProducerInterceptor.class.getName()); + props.setProperty(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, + CloseInterceptor.class.getName() + "," + MockProducerInterceptor.class.getName()); props.setProperty(MockProducerInterceptor.APPEND_STRING_PROP, "something"); MockProducerInterceptor.setThrowOnConfigExceptionThreshold(targetInterceptor); assertThrows(KafkaException.class, () -> - new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()) + new KafkaProducer<>(props, new StringSerializer(), new StringSerializer()) ); assertEquals(1, MockProducerInterceptor.CONFIG_COUNT.get()); assertEquals(1, MockProducerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, CloseInterceptor.CLOSE_COUNT.get()); } finally { MockProducerInterceptor.resetCounters(); + CloseInterceptor.resetCounters(); } } @@ -3204,4 +3207,38 @@ public void withPluginMetrics(PluginMetrics metrics) { metrics.addMetric(name, (Measurable) (config, now) -> VALUE); } } + + public static class CloseInterceptor implements ProducerInterceptor { + + public static final AtomicInteger CLOSE_COUNT = new AtomicInteger(0); + + @Override + public ProducerRecord onSend(ProducerRecord record) { + return null; + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception) { + ProducerInterceptor.super.onAcknowledgement(metadata, exception); + } + + @Override + public void onAcknowledgement(RecordMetadata metadata, Exception exception, Headers headers) { + ProducerInterceptor.super.onAcknowledgement(metadata, exception, headers); + } + + @Override + public void close() { + CLOSE_COUNT.incrementAndGet(); + } + + @Override + public void configure(Map configs) { + // no-op + } + + public static void resetCounters() { + CLOSE_COUNT.set(0); + } + } } diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index 15f5056bb2031..66f9bd4865731 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -32,7 +32,6 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs import org.slf4j.event.Level -import java.util.stream.Collectors import scala.jdk.CollectionConverters._ /** From fd8176c92d4425c616e47cb7dea8f494a1708b33 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 30 Aug 2025 12:58:21 +0800 Subject: [PATCH 66/78] fix Tls related test --- clients/src/test/java/org/apache/kafka/test/TestSslUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index b89aade368617..c04f54e0aa0e5 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -686,6 +686,7 @@ private Map buildJks() throws IOException, GeneralSecurityExcept sslConfigs.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, trustStorePassword); sslConfigs.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, "JKS"); sslConfigs.put(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, TrustManagerFactory.getDefaultAlgorithm()); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); From 93b36f1e17f1fd6c3637c31183aadd21d2f0e636 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 30 Aug 2025 12:59:55 +0800 Subject: [PATCH 67/78] fix Tls related test --- clients/src/test/java/org/apache/kafka/test/TestSslUtils.java | 1 + 1 file changed, 1 insertion(+) diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index c04f54e0aa0e5..dc8c144546c85 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -211,6 +211,7 @@ public static Map createSslConfig(String keyManagerAlgorithm, St sslConfigs.put(SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, keyManagerAlgorithm); sslConfigs.put(SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, trustManagerAlgorithm); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); From f4e657695c3c37456d5fdd988362bd769b6e7db1 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 31 Aug 2025 09:11:38 +0800 Subject: [PATCH 68/78] fix fail test --- .../common/network/SslTransportLayerTest.java | 5 +-- .../ssl/DefaultSslEngineFactoryTest.java | 12 ++++--- .../org/apache/kafka/test/TestSslUtils.java | 5 +-- .../runtime/rest/RestServerConfig.java | 33 ++++++++++++++++++- 4 files changed, 45 insertions(+), 10 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index d75028b1c80ca..72f130ca4e3ff 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -110,7 +110,8 @@ public Args(String tlsProtocol, boolean useInlinePem) throws Exception { this.useInlinePem = useInlinePem; sslConfigOverrides = new HashMap<>(); sslConfigOverrides.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocol); - sslConfigOverrides.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList(tlsProtocol)); + sslConfigOverrides.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(tlsProtocol)); + sslConfigOverrides.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); init(); } @@ -607,7 +608,7 @@ public void testInvalidKeyPassword(Args args) throws Exception { public void testTlsDefaults(Args args) throws Exception { args.sslServerConfigs = args.serverCertStores.getTrustingConfig(args.clientCertStores); args.sslClientConfigs = args.clientCertStores.getTrustingConfig(args.serverCertStores); - + args.sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, args.sslServerConfigs.get(SslConfigs.SSL_PROTOCOL_CONFIG)); assertEquals(SslConfigs.DEFAULT_SSL_PROTOCOL, args.sslClientConfigs.get(SslConfigs.SSL_PROTOCOL_CONFIG)); diff --git a/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java b/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java index 232d4d7327bf6..ed76495fa03a8 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/ssl/DefaultSslEngineFactoryTest.java @@ -202,6 +202,8 @@ public class DefaultSslEngineFactoryTest { public void setUp() { factory = sslEngineFactory(); configs.put(SslConfigs.SSL_PROTOCOL_CONFIG, "TLSv1.2"); + configs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); + configs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of()); } protected DefaultSslEngineFactory sslEngineFactory() { @@ -216,7 +218,7 @@ public void testPemTrustStoreConfigWithOneCert() throws Exception { KeyStore trustStore = factory.truststore(); List aliases = Collections.list(trustStore.aliases()); - assertEquals(Collections.singletonList("kafka0"), aliases); + assertEquals(List.of("kafka0"), aliases); assertNotNull(trustStore.getCertificate("kafka0"), "Certificate not loaded"); assertNull(trustStore.getKey("kafka0", null), "Unexpected private key"); } @@ -270,7 +272,7 @@ private void verifyPemKeyStoreConfig(String keyFileName, Password keyPassword) t KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not loaded"); assertNotNull(keyStore.getKey("kafka", keyPassword == null ? null : keyPassword.value().toCharArray()), "Private key not loaded"); @@ -284,7 +286,7 @@ public void testPemTrustStoreFile() throws Exception { KeyStore trustStore = factory.truststore(); List aliases = Collections.list(trustStore.aliases()); - assertEquals(Collections.singletonList("kafka0"), aliases); + assertEquals(List.of("kafka0"), aliases); assertNotNull(trustStore.getCertificate("kafka0"), "Certificate not found"); assertNull(trustStore.getKey("kafka0", null), "Unexpected private key"); } @@ -299,7 +301,7 @@ public void testPemKeyStoreFileNoKeyPassword() throws Exception { KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not loaded"); assertNotNull(keyStore.getKey("kafka", null), "Private key not loaded"); } @@ -314,7 +316,7 @@ public void testPemKeyStoreFileWithKeyPassword() throws Exception { KeyStore keyStore = factory.keystore(); List aliases = Collections.list(keyStore.aliases()); - assertEquals(Collections.singletonList("kafka"), aliases); + assertEquals(List.of("kafka"), aliases); assertNotNull(keyStore.getCertificate("kafka"), "Certificate not found"); assertNotNull(keyStore.getKey("kafka", KEY_PASSWORD.value().toCharArray()), "Private key not found"); } diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index dc8c144546c85..47a26aa697d1b 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -84,7 +84,6 @@ import java.security.cert.CertificateException; import java.security.cert.X509Certificate; import java.util.ArrayList; -import java.util.Collections; import java.util.Date; import java.util.Enumeration; import java.util.HashMap; @@ -703,7 +702,8 @@ private Map buildPem() throws IOException, GeneralSecurityExcept Map sslConfigs = new HashMap<>(); sslConfigs.put(SslConfigs.SSL_PROTOCOL_CONFIG, tlsProtocol); - sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Collections.singletonList(tlsProtocol)); + sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(tlsProtocol)); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); if (connectionMode != ConnectionMode.CLIENT || useClientCert) { KeyPair keyPair = generateKeyPair(algorithm); @@ -840,6 +840,7 @@ public static Map generateConfigsWithCertificateChains(String tl List enabledProtocols = new ArrayList<>(); enabledProtocols.add(tlsProtocol); sslConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, enabledProtocols); + sslConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, List.of()); return sslConfigs; } diff --git a/connect/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 9e7d2f7ffb11d..22623f74d6bcc 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 @@ -147,7 +147,7 @@ public static void addPublicConfig(ConfigDef configDef) { .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - ConfigDef.ValidList.anyNonDuplicateValues(true, true), + new AdminListenersValidator(), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } @@ -328,6 +328,37 @@ public String toString() { } } + private static class AdminListenersValidator implements ConfigDef.Validator { + @Override + public void ensureValid(String name, Object value) { + if (value == null) { + return; + } + + if (!(value instanceof List items)) { + throw new ConfigException("Invalid value type for admin.listeners (expected list)."); + } + + if (items.isEmpty()) { + return; + } + + for (Object item : items) { + if (!(item instanceof String)) { + throw new ConfigException("Invalid type for admin.listeners (expected String)."); + } + if (Utils.isBlank((String) item)) { + throw new ConfigException("Empty URL found when parsing admin.listeners list."); + } + } + } + + @Override + public String toString() { + return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443."; + } + } + private static class ResponseHttpHeadersValidator implements ConfigDef.Validator { @Override public void ensureValid(String name, Object value) { From 8e50b4b1455cc19c62dc0e982cc0830cfb0d61fc Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 31 Aug 2025 10:45:03 +0800 Subject: [PATCH 69/78] addressed by comments --- .../apache/kafka/connect/runtime/rest/util/SSLUtils.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java index 4a006c48e0a0a..83a175e8d5fff 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/util/SSLUtils.java @@ -123,7 +123,9 @@ protected static void configureSslContextFactoryTrustStore(SslContextFactory ssl @SuppressWarnings("unchecked") protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl, Map sslConfigValues) { List sslEnabledProtocols = (List) getOrDefault(sslConfigValues, SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, List.of(COMMA_WITH_WHITESPACE.split(SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS))); - ssl.setIncludeProtocols(sslEnabledProtocols.toArray(new String[0])); + + if (!sslEnabledProtocols.isEmpty()) + ssl.setIncludeProtocols(sslEnabledProtocols.toArray(new String[0])); String sslProvider = (String) sslConfigValues.get(SslConfigs.SSL_PROVIDER_CONFIG); if (sslProvider != null) @@ -132,7 +134,9 @@ protected static void configureSslContextFactoryAlgorithms(SslContextFactory ssl ssl.setProtocol((String) getOrDefault(sslConfigValues, SslConfigs.SSL_PROTOCOL_CONFIG, SslConfigs.DEFAULT_SSL_PROTOCOL)); List sslCipherSuites = (List) sslConfigValues.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); - ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[0])); + + if (!sslCipherSuites.isEmpty()) + ssl.setIncludeCipherSuites(sslCipherSuites.toArray(new String[0])); ssl.setKeyManagerFactoryAlgorithm((String) getOrDefault(sslConfigValues, SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM)); From 65877f2cf340ac8a378e8d1c75030087f8161bb0 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 31 Aug 2025 11:36:51 +0800 Subject: [PATCH 70/78] add upgrade.html section --- docs/upgrade.html | 24 +++++++++++++++++++ .../storage/internals/log/UnifiedLog.java | 6 ++--- 2 files changed, 27 insertions(+), 3 deletions(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 772484eb4ed5b..b351fe74baded 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -110,6 +110,30 @@
    Notable changes in 4
  • +
  • + Improvements have been made to the validation rules and default values of LIST-type configurations + (KIP-1161). +
      +
    • + LIST-type configurations now enforce stricter validation: +
        +
      • Null values are no longer accepted for most LIST-type configurations, except those that explicitly + allow a null default value or where a null value has a well-defined semantic meaning.
      • +
      • Duplicate entries within the same list are no longer permitted.
      • +
      • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined + semantic meaning.
      • +
      +
    • +
    • + Several configurations have been reclassified from STRING-type to LIST-type to better reflect their + intended use as comma-separated values. +
    • +
    • + Default values for certain configurations have been adjusted to ensure better consistency with related + settings. +
    • +
    +
  • Upgrading to 4.1.0

    diff --git a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java index a12e2b5963dc2..61b4b9d0edb2f 100644 --- a/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java +++ b/storage/src/main/java/org/apache/kafka/storage/internals/log/UnifiedLog.java @@ -1918,15 +1918,15 @@ public int deleteOldSegments() throws IOException { } else if (config().compact) { return deleteLogStartOffsetBreachedSegments(); } else { - // If cleanup.policy is empty and remote storage is enable, the local log segments will - // be cleaned based on the values of log.local.retention.bytes and log.local.retention.ms. + // If cleanup.policy is empty and remote storage is enabled, the local log segments will + // be cleaned based on the values of log.local.retention.bytes and log.local.retention.ms if (remoteLogEnabledAndRemoteCopyEnabled()) { return deleteLogStartOffsetBreachedSegments() + deleteRetentionSizeBreachedSegments() + deleteRetentionMsBreachedSegments(); } else { // If cleanup.policy is empty and remote storage is disabled, we should not delete any local - // log segments. + // log segments return deleteLogStartOffsetBreachedSegments(); } } From 66e9a5121f6d88b31709ed16089f3438a1be365c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 31 Aug 2025 11:39:44 +0800 Subject: [PATCH 71/78] resolve conflict --- docs/upgrade.html | 24 ++++++++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index 38ed0963af161..e57fac6723c9c 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -113,6 +113,30 @@
    Notable changes in 4
  • The num.replica.fetchers config has a new lower bound of 1.
  • +
  • + Improvements have been made to the validation rules and default values of LIST-type configurations + (KIP-1161). +
      +
    • + LIST-type configurations now enforce stricter validation: +
        +
      • Null values are no longer accepted for most LIST-type configurations, except those that explicitly + allow a null default value or where a null value has a well-defined semantic meaning.
      • +
      • Duplicate entries within the same list are no longer permitted.
      • +
      • Empty lists are no longer allowed, except in configurations where an empty list has a well-defined + semantic meaning.
      • +
      +
    • +
    • + Several configurations have been reclassified from STRING-type to LIST-type to better reflect their + intended use as comma-separated values. +
    • +
    • + Default values for certain configurations have been adjusted to ensure better consistency with related + settings. +
    • +
    +
  • Upgrading to 4.1.0

    From f454ec98bec24fa4db8f2a2cd57705366a5998c5 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 31 Aug 2025 15:54:42 +0800 Subject: [PATCH 72/78] add cleanup.policy section --- docs/upgrade.html | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/upgrade.html b/docs/upgrade.html index e57fac6723c9c..c7af07e0411ab 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -135,6 +135,11 @@
    Notable changes in 4 Default values for certain configurations have been adjusted to ensure better consistency with related settings. +
  • + The cleanup.policy is empty and remote.storage.enable is set to true, the + local log segments will be cleaned based on the values of log.local.retention.bytes and + log.local.retention.ms. +
  • From 847c0621c20d4c2d13143aaca2e70d4ea4b66425 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 2 Sep 2025 22:57:06 +0800 Subject: [PATCH 73/78] update the default value --- .../java/org/apache/kafka/server/config/ServerLogConfigs.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java index 4457b142d5a2f..5438a0a59cb6f 100644 --- a/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java +++ b/server-common/src/main/java/org/apache/kafka/server/config/ServerLogConfigs.java @@ -21,8 +21,6 @@ import org.apache.kafka.common.record.Records; import org.apache.kafka.server.record.BrokerCompressionType; -import java.util.List; - import static org.apache.kafka.server.config.ServerTopicConfigSynonyms.LOG_PREFIX; /** @@ -38,7 +36,7 @@ public class ServerLogConfigs { public static final String LOG_DIRS_CONFIG = LOG_PREFIX + "dirs"; public static final String LOG_DIR_CONFIG = LOG_PREFIX + "dir"; - public static final List LOG_DIR_DEFAULT = List.of("/tmp/kafka-logs"); + public static final String LOG_DIR_DEFAULT = "/tmp/kafka-logs"; public static final String LOG_DIR_DOC = "A comma-separated list of the directories where the log data is stored. (supplemental to " + LOG_DIRS_CONFIG + " property)"; public static final String LOG_DIRS_DOC = "A comma-separated list of the directories where the log data is stored. If not set, the value in " + LOG_DIR_CONFIG + " is used."; From d2d03b0d7f2983f29259494e25cad92e0459f851 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 4 Sep 2025 19:00:39 +0800 Subject: [PATCH 74/78] addressed by comment --- .../apache/kafka/common/config/ConfigDef.java | 24 +++++++++---- .../kafka/common/config/ConfigDefTest.java | 22 +++++++----- .../runtime/rest/RestServerConfig.java | 36 ++----------------- .../runtime/rest/RestServerConfigTest.java | 2 +- .../apache/kafka/tools/TopicCommandTest.java | 2 +- 5 files changed, 36 insertions(+), 50 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 466a187c52afc..8dd83ddd34b04 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 @@ -1042,17 +1042,27 @@ public void ensureValid(final String name, final Object value) { @SuppressWarnings("unchecked") List values = (List) value; if (!isEmptyAllowed && values.isEmpty()) { - String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString(); - throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); + String validValues = validString.validStrings.isEmpty() ? "any non-empty value" : validString.toString(); + throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validValues); } - if (Set.copyOf(values).size() != values.size()) { + + if (values.size() > 1 && Set.copyOf(values).size() != values.size()) { throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); } - if (validString.validStrings.isEmpty()) { - return; - } + + validateIndividualValues(name, values); + } + + private void validateIndividualValues(String name, List values) { + boolean hasValidStrings = !validString.validStrings.isEmpty(); + for (String string : values) { - validString.ensureValid(name, string); + if (string.isEmpty()) { + throw new ConfigException("Configuration '" + name + "' values must not be empty."); + } + if (hasValidStrings) { + validString.ensureValid(name, string); + } } } 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 61c2117e0984e..c6c2390b07c47 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 @@ -766,22 +766,28 @@ public void testListValidatorAnyNonDuplicateValues() { assertDoesNotThrow(() -> allowAnyNonDuplicateValues.ensureValid("test.config", null)); ConfigException exception1 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of("a", "a"))); assertEquals("Configuration 'test.config' values must not be duplicated.", exception1.getMessage()); + ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValues.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception2.getMessage()); ConfigDef.ValidList allowAnyNonDuplicateValuesAndNull = ConfigDef.ValidList.anyNonDuplicateValues(false, true); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "b", "c"))); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", null)); - ConfigException exception2 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of())); - assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception2.getMessage()); - ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "a"))); - assertEquals("Configuration 'test.config' values must not be duplicated.", exception3.getMessage()); + ConfigException exception3 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of())); + assertEquals("Configuration 'test.config' must not be empty. Valid values include: any non-empty value", exception3.getMessage()); + ConfigException exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception4.getMessage()); + ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndNull.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception5.getMessage()); ConfigDef.ValidList allowAnyNonDuplicateValuesAndEmptyList = ConfigDef.ValidList.anyNonDuplicateValues(true, false); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "b", "c"))); assertDoesNotThrow(() -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of())); - ConfigException exception4 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", null)); - assertEquals("Configuration 'test.config' values must not be null.", exception4.getMessage()); - ConfigException exception5 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "a"))); - assertEquals("Configuration 'test.config' values must not be duplicated.", exception5.getMessage()); + ConfigException exception6 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", null)); + assertEquals("Configuration 'test.config' values must not be null.", exception6.getMessage()); + ConfigException exception7 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of("a", "a"))); + assertEquals("Configuration 'test.config' values must not be duplicated.", exception7.getMessage()); + ConfigException exception8 = assertThrows(ConfigException.class, () -> allowAnyNonDuplicateValuesAndEmptyList.ensureValid("test.config", List.of(""))); + assertEquals("Configuration 'test.config' values must not be empty.", exception8.getMessage()); } @Test 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 22623f74d6bcc..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 @@ -85,7 +85,8 @@ public abstract class RestServerConfig extends AbstractConfig { private static final String ADMIN_LISTENERS_DOC = "List of comma-separated URIs the Admin REST API will listen on." + " The supported protocols are HTTP and HTTPS." + " An empty or blank string will disable this feature." + - " The default behavior is to use the regular listener (specified by the 'listeners' property)."; + " The default behavior is to use the regular listener (specified by the 'listeners' property)." + + " A comma-separated list of valid URLs, e.g., http://localhost:8080,https://localhost:8443."; public static final String ADMIN_LISTENERS_HTTPS_CONFIGS_PREFIX = "admin.listeners.https."; public static final String REST_EXTENSION_CLASSES_CONFIG = "rest.extension.classes"; @@ -147,7 +148,7 @@ public static void addPublicConfig(ConfigDef configDef) { .define(ADMIN_LISTENERS_CONFIG, ConfigDef.Type.LIST, null, - new AdminListenersValidator(), + ConfigDef.ValidList.anyNonDuplicateValues(true, true), ConfigDef.Importance.LOW, ADMIN_LISTENERS_DOC); } @@ -328,37 +329,6 @@ public String toString() { } } - private static class AdminListenersValidator implements ConfigDef.Validator { - @Override - public void ensureValid(String name, Object value) { - if (value == null) { - return; - } - - if (!(value instanceof List items)) { - throw new ConfigException("Invalid value type for admin.listeners (expected list)."); - } - - if (items.isEmpty()) { - return; - } - - for (Object item : items) { - if (!(item instanceof String)) { - throw new ConfigException("Invalid type for admin.listeners (expected String)."); - } - if (Utils.isBlank((String) item)) { - throw new ConfigException("Empty URL found when parsing admin.listeners list."); - } - } - } - - @Override - public String toString() { - return "List of comma-separated URLs, ex: http://localhost:8080,https://localhost:8443."; - } - } - private static class ResponseHttpHeadersValidator implements ConfigDef.Validator { @Override public void ensureValid(String name, Object value) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java index c9c91526625b0..58a2bd2e54294 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerConfigTest.java @@ -122,7 +122,7 @@ public void testAdminListenersNotAllowingEmptyStrings() { props.put(RestServerConfig.ADMIN_LISTENERS_CONFIG, "http://a.b:9999,"); ConfigException ce = assertThrows(ConfigException.class, () -> RestServerConfig.forPublic(null, props)); - assertTrue(ce.getMessage().contains(" admin.listeners")); + assertTrue(ce.getMessage().contains("admin.listeners")); } @Test diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index fe9028b8c4103..c6f1073edb56c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -885,7 +885,7 @@ public void testConfigPreservationAcrossPartitionAlteration(ClusterInstance clus public void testTopicDeletion(ClusterInstance clusterInstance) throws Exception { try (Admin adminClient = clusterInstance.admin(); TopicCommand.TopicService topicService = new TopicCommand.TopicService(adminClient)) { - String testTopicName = TestUtils.randomString(10); + String testTopicName = "testing"; adminClient.createTopics(List.of(new NewTopic(testTopicName, defaultNumPartitions, defaultReplicationFactor))); clusterInstance.waitTopicCreation(testTopicName, defaultNumPartitions); From c9457be0b4329ff0efdaac5f4037acbdcd30e58a Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 4 Sep 2025 19:58:54 +0800 Subject: [PATCH 75/78] fix sting cast --- .../main/java/org/apache/kafka/common/config/ConfigDef.java | 5 ++--- 1 file changed, 2 insertions(+), 3 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 8dd83ddd34b04..98bb2b413959b 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 @@ -1030,7 +1030,6 @@ public static ValidList in(boolean isEmptyAllowed, String... validStrings) { return new ValidList(List.of(validStrings), isEmptyAllowed, false); } - @Override public void ensureValid(final String name, final Object value) { if (value == null) { if (isNullAllowed) @@ -1042,8 +1041,8 @@ public void ensureValid(final String name, final Object value) { @SuppressWarnings("unchecked") List values = (List) value; if (!isEmptyAllowed && values.isEmpty()) { - String validValues = validString.validStrings.isEmpty() ? "any non-empty value" : validString.toString(); - throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validValues); + String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString(); + throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); } if (values.size() > 1 && Set.copyOf(values).size() != values.size()) { From 34db3921000d170c24b2faaecad24148c1e17cbd Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 4 Sep 2025 20:00:36 +0800 Subject: [PATCH 76/78] add override --- .../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 98bb2b413959b..96f10099049d1 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 @@ -1030,6 +1030,7 @@ public static ValidList in(boolean isEmptyAllowed, String... validStrings) { return new ValidList(List.of(validStrings), isEmptyAllowed, false); } + @Override public void ensureValid(final String name, final Object value) { if (value == null) { if (isNullAllowed) From b5b50534713ef01b24d4558512f36971fb4c7f4e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 4 Sep 2025 20:20:48 +0800 Subject: [PATCH 77/78] fix object cast to string --- .../apache/kafka/common/config/ConfigDef.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 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 96f10099049d1..ad5b74e194bad 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 @@ -1040,7 +1040,7 @@ public void ensureValid(final String name, final Object value) { } @SuppressWarnings("unchecked") - List values = (List) value; + List values = (List) value; if (!isEmptyAllowed && values.isEmpty()) { String validString = this.validString.validStrings.isEmpty() ? "any non-empty value" : this.validString.toString(); throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); @@ -1053,15 +1053,18 @@ public void ensureValid(final String name, final Object value) { validateIndividualValues(name, values); } - private void validateIndividualValues(String name, List values) { + private void validateIndividualValues(String name, List values) { boolean hasValidStrings = !validString.validStrings.isEmpty(); - for (String string : values) { - if (string.isEmpty()) { - throw new ConfigException("Configuration '" + name + "' values must not be empty."); - } - if (hasValidStrings) { - validString.ensureValid(name, string); + for (Object value : values) { + if (value instanceof String) { + String string = (String) value; + if (string.isEmpty()) { + throw new ConfigException("Configuration '" + name + "' values must not be empty."); + } + if (hasValidStrings) { + validString.ensureValid(name, value); + } } } } From 43c415578a28116fd71ce45188e39ae24662b090 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 4 Sep 2025 20:24:26 +0800 Subject: [PATCH 78/78] update the valid logic --- .../src/main/java/org/apache/kafka/common/config/ConfigDef.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ad5b74e194bad..ee2f8c2cfd951 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 @@ -1046,7 +1046,7 @@ public void ensureValid(final String name, final Object value) { throw new ConfigException("Configuration '" + name + "' must not be empty. Valid values include: " + validString); } - if (values.size() > 1 && Set.copyOf(values).size() != values.size()) { + if (Set.copyOf(values).size() != values.size()) { throw new ConfigException("Configuration '" + name + "' values must not be duplicated."); }