This is an automated email from the ASF dual-hosted git repository.
mimaison pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new b9198365511 KAFKA-17662: config.providers configuration missing from
the docs (#18930)
b9198365511 is described below
commit b9198365511903b7a6515b02ce45629f3e5d5241
Author: Ken Huang <[email protected]>
AuthorDate: Fri Jun 27 20:13:55 2025 +0800
KAFKA-17662: config.providers configuration missing from the docs (#18930)
Ensure the config.providers configuration is documented for all
components supporting it
Reviewers: Mickael Maison <[email protected]>, Greg Harris
<[email protected]>, Matthias J. Sax <[email protected]>
---
.../java/org/apache/kafka/clients/admin/AdminClientConfig.java | 8 +++++++-
.../java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 7 ++++++-
.../java/org/apache/kafka/clients/producer/ProducerConfig.java | 8 +++++++-
.../main/java/org/apache/kafka/common/config/AbstractConfig.java | 4 ++++
.../org/apache/kafka/connect/mirror/MirrorConnectorConfig.java | 5 +++++
.../main/java/org/apache/kafka/connect/runtime/WorkerConfig.java | 8 ++------
core/src/main/scala/kafka/server/DynamicBrokerConfig.scala | 3 ++-
core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala | 3 ++-
.../main/java/org/apache/kafka/server/config/ServerConfigs.java | 5 +++++
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java | 5 +++++
10 files changed, 45 insertions(+), 11 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 a87af6be154..d883ca1450e 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
@@ -30,6 +30,7 @@ import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.Utils;
+import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -280,7 +281,12 @@ public class AdminClientConfig extends AbstractConfig {
DEFAULT_METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS,
atLeast(0),
Importance.LOW,
-
METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC);
+
METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC)
+ .define(CONFIG_PROVIDERS_CONFIG,
+ ConfigDef.Type.LIST,
+ List.of(),
+ ConfigDef.Importance.LOW,
+ CONFIG_PROVIDERS_DOC);
}
@Override
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 272fd3d22b4..2e0c186186d 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
@@ -702,7 +702,12 @@ public class ConsumerConfig extends AbstractConfig {
ShareAcknowledgementMode.IMPLICIT.name(),
new
ShareAcknowledgementMode.Validator(),
Importance.MEDIUM,
-
ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_DOC);
+
ConsumerConfig.SHARE_ACKNOWLEDGEMENT_MODE_DOC)
+ .define(CONFIG_PROVIDERS_CONFIG,
+ ConfigDef.Type.LIST,
+ List.of(),
+ ConfigDef.Importance.LOW,
+ CONFIG_PROVIDERS_DOC);
}
@Override
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 283f021017d..e28b5d30812 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
@@ -37,6 +37,7 @@ import org.slf4j.LoggerFactory;
import java.util.Collections;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
@@ -549,7 +550,12 @@ public class ProducerConfig extends AbstractConfig {
CommonClientConfigs.DEFAULT_METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS,
atLeast(0),
Importance.LOW,
-
CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC);
+
CommonClientConfigs.METADATA_RECOVERY_REBOOTSTRAP_TRIGGER_MS_DOC)
+ .define(CONFIG_PROVIDERS_CONFIG,
+ ConfigDef.Type.LIST,
+ List.of(),
+ ConfigDef.Importance.LOW,
+ CONFIG_PROVIDERS_DOC);
}
@Override
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 9188b1687a8..e271cd99c4c 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -65,6 +65,10 @@ public class AbstractConfig {
public static final String AUTOMATIC_CONFIG_PROVIDERS_PROPERTY =
"org.apache.kafka.automatic.config.providers";
public static final String CONFIG_PROVIDERS_CONFIG = "config.providers";
+ public static final String CONFIG_PROVIDERS_DOC =
+ "Comma-separated alias names for classes implementing the
<code>ConfigProvider</code> interface. " +
+ "This enables loading configuration data (such as passwords, API
keys, and other credentials) from external " +
+ "sources. For example, see <a
href=\"https://kafka.apache.org/documentation/#config_providers\">Configuration
Providers</a>.";
private static final String CONFIG_PROVIDERS_PARAM = ".param.";
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 920f1d93d04..849f40c1fce 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
@@ -320,6 +320,11 @@ public abstract class MirrorConnectorConfig extends
AbstractConfig {
in(Utils.enumOptions(SecurityProtocol.class)),
ConfigDef.Importance.MEDIUM,
CommonClientConfigs.SECURITY_PROTOCOL_DOC)
+ .define(CONFIG_PROVIDERS_CONFIG,
+ ConfigDef.Type.LIST,
+ List.of(),
+ ConfigDef.Importance.LOW,
+ CONFIG_PROVIDERS_DOC)
.withClientSslSupport()
.withClientSaslSupport();
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 a68cdb4ea03..e859accb527 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
@@ -154,12 +154,8 @@ public class WorkerConfig extends AbstractConfig {
+ "* " + SERVICE_LOAD + ": Discover plugins only by ServiceLoader.
Faster startup than other modes. "
+ "Plugins which are not discoverable by ServiceLoader may not be
usable.";
- public static final String CONFIG_PROVIDERS_CONFIG = "config.providers";
- protected static final String CONFIG_PROVIDERS_DOC =
- "Comma-separated names of <code>ConfigProvider</code> classes,
loaded and used "
- + "in the order specified. Implementing the interface "
- + "<code>ConfigProvider</code> allows you to replace variable
references in connector configurations, "
- + "such as for externalized secrets. ";
+ public static final String CONFIG_PROVIDERS_CONFIG =
AbstractConfig.CONFIG_PROVIDERS_CONFIG;
+ protected static final String CONFIG_PROVIDERS_DOC =
AbstractConfig.CONFIG_PROVIDERS_DOC;
public static final String CONNECTOR_CLIENT_POLICY_CLASS_CONFIG =
"connector.client.config.override.policy";
public static final String CONNECTOR_CLIENT_POLICY_CLASS_DOC =
diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 74ae428deae..c64218246b3 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -98,7 +98,8 @@ object DynamicBrokerConfig {
DynamicListenerConfig.ReconfigurableConfigs ++
SocketServer.ReconfigurableConfigs ++
DynamicProducerStateManagerConfig ++
- DynamicRemoteLogConfig.ReconfigurableConfigs
+ DynamicRemoteLogConfig.ReconfigurableConfigs ++
+ Set(AbstractConfig.CONFIG_PROVIDERS_CONFIG)
private val ClusterLevelListenerConfigs =
Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG,
SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG,
SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG)
private val PerBrokerConfigs = (DynamicSecurityConfigs ++
DynamicListenerConfig.ReconfigurableConfigs).diff(
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 835fc093f0f..43384a64789 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -23,7 +23,7 @@ import java.util.{Arrays, Collections, Properties}
import kafka.utils.TestUtils.assertBadConfigContainingMessage
import kafka.utils.{CoreUtils, TestUtils}
import org.apache.kafka.common.{Endpoint, Node}
-import org.apache.kafka.common.config.{ConfigException, SaslConfigs,
SecurityConfig, SslConfigs, TopicConfig}
+import org.apache.kafka.common.config.{AbstractConfig, ConfigException,
SaslConfigs, SecurityConfig, SslConfigs, TopicConfig}
import org.apache.kafka.common.metrics.Sensor
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.record.{CompressionType, Records}
@@ -782,6 +782,7 @@ class KafkaConfigTest {
KafkaConfig.configNames.foreach { name =>
name match {
+ case AbstractConfig.CONFIG_PROVIDERS_CONFIG => // ignore string
case ServerConfigs.BROKER_ID_CONFIG =>
assertPropertyInvalid(baseProperties, name, "not_a_number")
case ServerConfigs.NUM_IO_THREADS_CONFIG =>
assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
case ServerConfigs.BACKGROUND_THREADS_CONFIG =>
assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
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 fdbf40239f4..4910f00016b 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
@@ -23,6 +23,10 @@ import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.record.BrokerCompressionType;
+import java.util.List;
+
+import static
org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CONFIG;
+import static
org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_DOC;
import static org.apache.kafka.common.config.ConfigDef.Importance.HIGH;
import static org.apache.kafka.common.config.ConfigDef.Importance.LOW;
import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM;
@@ -129,6 +133,7 @@ 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)
/************* 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)
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 944446b4bec..1043e5fb6f0 100644
--- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
+++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
@@ -1086,6 +1086,11 @@ public class StreamsConfig extends AbstractConfig {
atLeast(0),
Importance.LOW,
COMMIT_INTERVAL_MS_DOC)
+ .define(CONFIG_PROVIDERS_CONFIG,
+ Type.LIST,
+ List.of(),
+ Importance.LOW,
+ CONFIG_PROVIDERS_DOC)
.define(ENABLE_METRICS_PUSH_CONFIG,
Type.BOOLEAN,
true,