kafka/docs/configuration.html

456 lines
31 KiB
HTML
Raw Permalink Blame History

This file contains ambiguous Unicode characters

This file contains Unicode characters that might be confused with other characters. If you think that this is intentional, you can safely ignore this warning. Use the Escape button to reveal them.

<!--
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.
-->
<script id="configuration-template" type="text/x-handlebars-template">
Kafka uses key-value pairs in the <a href="https://en.wikipedia.org/wiki/.properties">property file format</a> for configuration. These values can be supplied either from a file or programmatically.
<h3 class="anchor-heading"><a id="brokerconfigs" class="anchor-link"></a><a href="#brokerconfigs">3.1 Broker Configs</a></h3>
The essential configurations are the following:
<ul>
<li><code>node.id</code>
<li><code>log.dirs</code>
<li><code>process.roles</code>
<li><code>controller.quorum.bootstrap.servers</code>
</ul>
Topic configurations and defaults are discussed in more detail <a href="#topicconfigs">below</a>.
<!--#include virtual="generated/kafka_config.html" -->
<p>More details about broker configuration can be found in the scala class <code>kafka.server.KafkaConfig</code>.</p>
<h4 class="anchor-heading"><a id="dynamicbrokerconfigs" class="anchor-link"></a><a href="#dynamicbrokerconfigs">3.1.1 Updating Broker Configs</a></h4>
From Kafka version 1.1 onwards, some of the broker configs can be updated without restarting the broker. See the
<code>Dynamic Update Mode</code> column in <a href="#brokerconfigs">Broker Configs</a> for the update mode of each broker config.
<ul>
<li><code>read-only</code>: Requires a broker restart for update</li>
<li><code>per-broker</code>: May be updated dynamically for each broker</li>
<li><code>cluster-wide</code>: May be updated dynamically as a cluster-wide default. May also be updated as a per-broker value for testing.</li>
</ul>
To alter the current broker configs for broker id 0 (for example, the number of log cleaner threads):
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --alter --add-config log.cleaner.threads=2</code></pre>
To describe the current dynamic broker configs for broker id 0:
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --describe</code></pre>
To delete a config override and revert to the statically configured or default value for broker id 0 (for example,
the number of log cleaner threads):
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-name 0 --alter --delete-config log.cleaner.threads</code></pre>
Some configs may be configured as a cluster-wide default to maintain consistent values across the whole cluster. All brokers
in the cluster will process the cluster default update. For example, to update log cleaner threads on all brokers:
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-default --alter --add-config log.cleaner.threads=2</code></pre>
To describe the currently configured dynamic cluster-wide default configs:
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type brokers --entity-default --describe</code></pre>
All configs that are configurable at cluster level may also be configured at per-broker level (e.g. for testing).
If a config value is defined at different levels, the following order of precedence is used:
<ul>
<li>Dynamic per-broker config stored in the metadata log</li>
<li>Dynamic cluster-wide default config stored in the metadata log</li>
<li>Static broker config from <code>server.properties</code></li>
<li>Kafka default, see <a href="#brokerconfigs">broker configs</a></li>
</ul>
<h5>Updating SSL Keystore of an Existing Listener</h5>
Brokers may be configured with SSL keystores with short validity periods to reduce the risk of compromised certificates.
Keystores may be updated dynamically without restarting the broker. The config name must be prefixed with the listener prefix
<code>listener.name.{listenerName}.</code> so that only the keystore config of a specific listener is updated.
The following configs may be updated in a single alter request at per-broker level:
<ul>
<li><code>ssl.keystore.type</code></li>
<li><code>ssl.keystore.location</code></li>
<li><code>ssl.keystore.password</code></li>
<li><code>ssl.key.password</code></li>
</ul>
If the listener is the inter-broker listener, the update is allowed only if the new keystore is trusted by the truststore
configured for that listener. For other listeners, no trust validation is performed on the keystore by the broker. Certificates
must be signed by the same certificate authority that signed the old certificate to avoid any client authentication failures.
<h5>Updating SSL Truststore of an Existing Listener</h5>
Broker truststores may be updated dynamically without restarting the broker to add or remove certificates.
Updated truststore will be used to authenticate new client connections. The config name must be prefixed with the
listener prefix <code>listener.name.{listenerName}.</code> so that only the truststore config of a specific listener
is updated. The following configs may be updated in a single alter request at per-broker level:
<ul>
<li><code>ssl.truststore.type</code></li>
<li><code>ssl.truststore.location</code></li>
<li><code>ssl.truststore.password</code></li>
</ul>
If the listener is the inter-broker listener, the update is allowed only if the existing keystore for that listener is trusted by
the new truststore. For other listeners, no trust validation is performed by the broker before the update. Removal of CA certificates
used to sign client certificates from the new truststore can lead to client authentication failures.
<h5>Updating Default Topic Configuration</h5>
Default topic configuration options used by brokers may be updated without broker restart. The configs are applied to topics
without a topic config override for the equivalent per-topic config. One or more of these configs may be overridden at
cluster-default level used by all brokers.
<ul>
<li><code>log.segment.bytes</code></li>
<li><code>log.roll.ms</code></li>
<li><code>log.roll.hours</code></li>
<li><code>log.roll.jitter.ms</code></li>
<li><code>log.roll.jitter.hours</code></li>
<li><code>log.index.size.max.bytes</code></li>
<li><code>log.flush.interval.messages</code></li>
<li><code>log.flush.interval.ms</code></li>
<li><code>log.retention.bytes</code></li>
<li><code>log.retention.ms</code></li>
<li><code>log.retention.minutes</code></li>
<li><code>log.retention.hours</code></li>
<li><code>log.index.interval.bytes</code></li>
<li><code>log.cleaner.delete.retention.ms</code></li>
<li><code>log.cleaner.min.compaction.lag.ms</code></li>
<li><code>log.cleaner.max.compaction.lag.ms</code></li>
<li><code>log.cleaner.min.cleanable.ratio</code></li>
<li><code>log.cleanup.policy</code></li>
<li><code>log.segment.delete.delay.ms</code></li>
<li><code>unclean.leader.election.enable</code></li>
<li><code>min.insync.replicas</code></li>
<li><code>max.message.bytes</code></li>
<li><code>compression.type</code></li>
<li><code>log.preallocate</code></li>
<li><code>log.message.timestamp.type</code></li>
</ul>
<h5>Updating Log Cleaner Configs</h5>
Log cleaner configs may be updated dynamically at cluster-default level used by all brokers. The changes take effect
on the next iteration of log cleaning. One or more of these configs may be updated:
<ul>
<li><code>log.cleaner.threads</code></li>
<li><code>log.cleaner.io.max.bytes.per.second</code></li>
<li><code>log.cleaner.dedupe.buffer.size</code></li>
<li><code>log.cleaner.io.buffer.size</code></li>
<li><code>log.cleaner.io.buffer.load.factor</code></li>
<li><code>log.cleaner.backoff.ms</code></li>
</ul>
<h5>Updating Thread Configs</h5>
The size of various thread pools used by the broker may be updated dynamically at cluster-default level used by all brokers.
Updates are restricted to the range <code>currentSize / 2</code> to <code>currentSize * 2</code> to ensure that config updates are
handled gracefully.
<ul>
<li><code>num.network.threads</code></li>
<li><code>num.io.threads</code></li>
<li><code>num.replica.fetchers</code></li>
<li><code>num.recovery.threads.per.data.dir</code></li>
<li><code>log.cleaner.threads</code></li>
<li><code>background.threads</code></li>
<li><code>remote.log.reader.threads</code></li>
<li><code>remote.log.manager.copier.thread.pool.size</code></li>
<li><code>remote.log.manager.expiration.thread.pool.size</code></li>
<li><code>remote.log.manager.follower.thread.pool.size</code></li>
</ul>
<h5>Updating ConnectionQuota Configs</h5>
The maximum number of connections allowed for a given IP/host by the broker may be updated dynamically at cluster-default level used by all brokers.
The changes will apply for new connection creations and the existing connections count will be taken into account by the new limits.
<ul>
<li><code>max.connections.per.ip</code></li>
<li><code>max.connections.per.ip.overrides</code></li>
</ul>
<h5>Adding and Removing Listeners</h5>
<p>Listeners may be added or removed dynamically. When a new listener is added, security configs of the listener must be provided
as listener configs with the listener prefix <code>listener.name.{listenerName}.</code>. If the new listener uses SASL,
the JAAS configuration of the listener must be provided using the JAAS configuration property <code>sasl.jaas.config</code>
with the listener and mechanism prefix. See <a href="#security_jaas_broker">JAAS configuration for Kafka brokers</a> for details.</p>
<p>In Kafka version 1.1.x, the listener used by the inter-broker listener may not be updated dynamically. To update the inter-broker
listener to a new listener, the new listener may be added on all brokers without restarting the broker. A rolling restart is then
required to update <code>inter.broker.listener.name</code>.</p>
In addition to all the security configs of new listeners, the following configs may be updated dynamically at per-broker level:
<ul>
<li><code>listeners</code></li>
<li><code>advertised.listeners</code></li>
<li><code>listener.security.protocol.map</code></li>
</ul>
Inter-broker listener must be configured using the static broker configuration <code>inter.broker.listener.name</code>
or <code>security.inter.broker.protocol</code>.
<h3 class="anchor-heading"><a id="topicconfigs" class="anchor-link"></a><a href="#topicconfigs">3.2 Topic Configs</a></h3>
Configurations pertinent to topics have both a server default as well an optional per-topic override. If no per-topic configuration is given the server default is used. The override can be set at topic creation time by giving one or more <code>--config</code> options. This example creates a topic named <i>my-topic</i> with a custom max message size and flush rate:
<pre><code class="language-bash">$ bin/kafka-topics.sh --bootstrap-server localhost:9092 --create --topic my-topic --partitions 1 \
--replication-factor 1 --config max.message.bytes=64000 --config flush.messages=1</code></pre>
Overrides can also be changed or set later using the alter configs command. This example updates the max message size for <i>my-topic</i>:
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic
--alter --add-config max.message.bytes=128000</code></pre>
To check overrides set on the topic you can do
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic --describe</code></pre>
To remove an override you can do
<pre><code class="language-bash">$ bin/kafka-configs.sh --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic
--alter --delete-config max.message.bytes</code></pre>
Below is the topic configuration. The server's default configuration for this property is given under the Server Default Property heading. A given server default config value only applies to a topic if it does not have an explicit topic config override.
<!--#include virtual="generated/topic_config.html" -->
<h3 class="anchor-heading"><a id="groupconfigs" class="anchor-link"></a><a href="#groupconfigs">3.3 Group Configs</a></h3>
Below is the group configuration:
<!--#include virtual="generated/group_config.html" -->
<h3 class="anchor-heading"><a id="producerconfigs" class="anchor-link"></a><a href="#producerconfigs">3.4 Producer Configs</a></h3>
Below is the producer configuration:
<!--#include virtual="generated/producer_config.html" -->
<h3 class="anchor-heading"><a id="consumerconfigs" class="anchor-link"></a><a href="#consumerconfigs">3.5 Consumer Configs</a></h3>
Below is the consumer and share consumer configuration:
<!--#include virtual="generated/consumer_config.html" -->
<h3 class="anchor-heading"><a id="connectconfigs" class="anchor-link"></a><a href="#connectconfigs">3.6 Kafka Connect Configs</a></h3>
Below is the Kafka Connect framework configuration.
<!--#include virtual="generated/connect_config.html" -->
<h4 class="anchor-heading"><a id="sourceconnectconfigs" class="anchor-link"></a><a href="#sourceconnectconfigs">3.6.1 Source Connector Configs</a></h4>
Below is the source connector configuration.
<!--#include virtual="generated/source_connector_config.html" -->
<h4 class="anchor-heading"><a id="sinkconnectconfigs" class="anchor-link"></a><a href="#sinkconnectconfigs">3.6.2 Sink Connector Configs</a></h4>
Below is the sink connector configuration.
<!--#include virtual="generated/sink_connector_config.html" -->
<h3 class="anchor-heading"><a id="streamsconfigs" class="anchor-link"></a><a href="#streamsconfigs">3.7 Kafka Streams Configs</a></h3>
Below is the Kafka Streams client library configuration.
<!--#include virtual="generated/streams_config.html" -->
<h3 class="anchor-heading"><a id="adminclientconfigs" class="anchor-link"></a><a href="#adminclientconfigs">3.8 Admin Configs</a></h3>
Below is the Kafka Admin client library configuration.
<!--#include virtual="generated/admin_client_config.html" -->
<h3 class="anchor-heading"><a id="mirrormakerconfigs" class="anchor-link"></a><a href="#mirrormakerconfigs">3.9 MirrorMaker Configs</a></h3>
Below is the configuration of the connectors that make up MirrorMaker 2.
<h4 class="anchor-heading"><a id="mirrormakercommonconfigs" class="anchor-link"></a><a href="#mirrormakercommonconfigs">3.9.1 MirrorMaker Common Configs</a></h4>
Below is the common configuration that applies to all three connectors.
<!--#include virtual="generated/mirror_connector_config.html" -->
<h4 class="anchor-heading"><a id="mirrormakersourceconfigs" class="anchor-link"></a><a href="#mirrormakersourceconfigs">3.9.2 MirrorMaker Source Configs</a></h4>
Below is the configuration of MirrorMaker 2 source connector for replicating topics.
<!--#include virtual="generated/mirror_source_config.html" -->
<h4 class="anchor-heading"><a id="mirrormakercheckpointconfigs" class="anchor-link"></a><a href="#mirrormakercheckpointconfigs">3.9.3 MirrorMaker Checkpoint Configs</a></h4>
Below is the configuration of MirrorMaker 2 checkpoint connector for emitting consumer offset checkpoints.
<!--#include virtual="generated/mirror_checkpoint_config.html" -->
<h4 class="anchor-heading"><a id="mirrormakerheartbeatconfigs" class="anchor-link"></a><a href="#mirrormakerheartbeatconfigs">3.9.4 MirrorMaker HeartBeat Configs</a></h4>
Below is the configuration of MirrorMaker 2 heartbeat connector for checking connectivity between connectors and clusters.
<!--#include virtual="generated/mirror_heartbeat_config.html" -->
<h3 class="anchor-heading"><a id="systemproperties" class="anchor-link"></a><a href="#systemproperties">3.10 System Properties</a></h3>
Kafka supports some configuration that can be enabled through Java system properties. System properties are usually set by passing the -D flag to the Java virtual machine in which Kafka components are running.
Below are the supported system properties.
<ul class="config-list">
<li>
<h4><a id="org.apache.kafka.sasl.oauthbearer.allowed.files"></a><a id="systemproperties_org.apache.kafka.sasl.oauthbearer.allowed.files" href="#systemproperties_org.apache.kafka.sasl.oauthbearer.allowed.files">org.apache.kafka.sasl.oauthbearer.allowed.files</a></h4>
<p>This system property is used to determine which files, if any, are allowed to be read by the SASL OAUTHBEARER plugin. This property accepts comma-separated list of files. By default the value is an empty list.
<p>If users want to enable some files, users need to explicitly set the system property like below.
<p><pre><code class="language-bash">-Dorg.apache.kafka.sasl.oauthbearer.allowed.files=/tmp/token,/tmp/private_key.pem</code></pre>
<table><tbody>
<tr><th>Since:</th><td>4.1.0</td></tr>
<tr><th>Default Value:</th><td></td></tr>
</tbody></table>
</li>
<li>
<h4><a id="org.apache.kafka.sasl.oauthbearer.allowed.urls"></a><a id="systemproperties_org.apache.kafka.sasl.oauthbearer.allowed.urls" href="#systemproperties_org.apache.kafka.sasl.oauthbearer.allowed.urls">org.apache.kafka.sasl.oauthbearer.allowed.urls</a></h4>
<p>This system property is used to set the allowed URLs as SASL OAUTHBEARER token or jwks endpoints. This property accepts comma-separated list of URLs. By default the value is an empty list.
<p>If users want to enable some URLs, users need to explicitly set the system property like below.
<p><pre><code class="language-bash">-Dorg.apache.kafka.sasl.oauthbearer.allowed.urls=https://www.example.com,file:///tmp/token</code></pre>
<table><tbody>
<tr><th>Since:</th><td>4.0.0</td></tr>
<tr><th>Default Value:</th><td></td></tr>
</tbody></table>
</li>
<li>
<h4><a id="org.apache.kafka.disallowed.login.modules"></a><a id="systemproperties_org.apache.kafka.disallowed.login.modules" href="#systemproperties_org.apache.kafka.disallowed.login.modules">org.apache.kafka.disallowed.login.modules</a></h4>
<p>This system property is used to disable the problematic login modules usage in SASL JAAS configuration. This property accepts comma-separated list of loginModule names. By default <b>com.sun.security.auth.module.JndiLoginModule</b> and <b>com.sun.security.auth.module.LdapLoginModule</b> loginModule is disabled.
<p>If users want to enable JndiLoginModule or LdapLoginModule, users need to explicitly reset the system property like below. We advise the users to validate configurations and only allow trusted JNDI configurations. For more details <a href="https://kafka.apache.org/cve-list#CVE-2023-25194">CVE-2023-25194</a>.
<p><pre><code class="language-bash">-Dorg.apache.kafka.disallowed.login.modules=</code></pre>
<p>To disable more loginModules, update the system property with comma-separated loginModule names. Make sure to explicitly add <b>JndiLoginModule</b> module name to the comma-separated list like below.
<p><pre><code class="language-bash">-Dorg.apache.kafka.disallowed.login.modules=com.sun.security.auth.module.JndiLoginModule,com.ibm.security.auth.module.LdapLoginModule,com.ibm.security.auth.module.Krb5LoginModule</code></pre>
<p>The configuration is deprecated and will be removed in a future release. Please use <b>org.apache.kafka.allowed.login.modules</b> instead.
<table><tbody>
<tr><th>Since:</th><td>3.4.0</td></tr>
<tr><th>Deprecated:</th><td>4.2.0</td></tr>
<tr><th>Default Value:</th><td>com.sun.security.auth.module.JndiLoginModule,com.sun.security.auth.module.LdapLoginModule</td></tr>
</tbody></table>
</li>
<li>
<h4><a id="org.apache.kafka.allowed.login.modules"></a><a id="systemproperties_org.apache.kafka.allowed.login.modules" href="#systemproperties_org.apache.kafka.allowed.login.modules">org.apache.kafka.allowed.login.modules</a></h4>
<p>This system property is used to explicitly allow specific login modules in SASL JAAS configuration. It accepts a comma-separated list of login module class names. This property provides a stricter, allowed-list-based alternative to the deprecated <code>org.apache.kafka.disallowed.login.modules</code> property.
It is recommended to use this property to improve the security of JAAS configurations.
<p>If both properties are set, <code>org.apache.kafka.allowed.login.modules</code> takes precedence.</p>
<table><tbody>
<tr><th>Since:</th><td>4.2.0</td></tr>
<tr><th>Default Value:</th></tr>
</tbody></table>
</li>
<li>
<h4><a id="org.apache.kafka.automatic.config.providers"></a><a id="systemproperties_org.apache.kafka.automatic.config.providers" href="#systemproperties_org.apache.kafka.automatic.config.providers">org.apache.kafka.automatic.config.providers</a></h4>
<p>This system property controls the automatic loading of ConfigProvider implementations in Apache Kafka. ConfigProviders are used to dynamically supply configuration values from sources such as files, directories, or environment variables. This property accepts a comma-separated list of ConfigProvider names. By default, all built-in ConfigProviders are enabled, including <b>FileConfigProvider</b>, <b>DirectoryConfigProvider</b>, and <b>EnvVarConfigProvider</b>.</p>
<p>If users want to disable all automatic ConfigProviders, they need to explicitly set the system property as shown below. Disabling automatic ConfigProviders is recommended in environments where configuration data comes from untrusted sources or where increased security is required. For more details, see <a href="https://kafka.apache.org/cve-list#CVE-2024-31141">CVE-2024-31141</a>.</p>
<pre><code class="language-bash">-Dorg.apache.kafka.automatic.config.providers=none</code></pre>
<p>To allow specific ConfigProviders, update the system property with a comma-separated list of fully qualified ConfigProvider class names. For example, to enable only the <b>EnvVarConfigProvider</b>, set the property as follows:</p>
<pre><code class="language-bash">-Dorg.apache.kafka.automatic.config.providers=org.apache.kafka.common.config.provider.EnvVarConfigProvider</code></pre>
<p>To use multiple ConfigProviders, include their names in a comma-separated list as shown below:</p>
<pre><code class="language-bash">-Dorg.apache.kafka.automatic.config.providers=org.apache.kafka.common.config.provider.FileConfigProvider,org.apache.kafka.common.config.provider.EnvVarConfigProvider</code></pre>
<table>
<tbody>
<tr><th>Since:</th><td>3.8.0</td></tr>
<tr><th>Default Value:</th><td>All built-in ConfigProviders are enabled</td></tr>
</tbody>
</table>
</li>
</ul>
<h3 class="anchor-heading"><a id="tieredstorageconfigs" class="anchor-link"></a><a href="#tieredstorageconfigs">3.11 Tiered Storage Configs</a></h3>
Below is the Tiered Storage configuration.
<!--#include virtual="generated/remote_log_manager_config.html" -->
<!--#include virtual="generated/remote_log_metadata_manager_config.html" -->
<h3 class="anchor-heading">
<a id="config_providers" class="anchor-link"></a>
<a href="#config_providers">3.12 Configuration Providers</a>
</h3>
<p>
Use configuration providers to load configuration data from external sources. This might include sensitive information, such as passwords, API keys, or other credentials.
</p>
<p>You have the following options:</p>
<ul>
<li>
Use a custom provider by creating a class implementing the
<a href="/{{version}}/javadoc/org/apache/kafka/common/config/provider/ConfigProvider.html"><code>ConfigProvider</code></a> interface and packaging it into a JAR file.
</li>
<li>Use a built-in provider:</li>
<ul>
<li><a href="/{{version}}/javadoc/org/apache/kafka/common/config/provider/DirectoryConfigProvider.html"><code>DirectoryConfigProvider</code></a></li>
<li><a href="/{{version}}/javadoc/org/apache/kafka/common/config/provider/EnvVarConfigProvider.html"><code>EnvVarConfigProvider</code></a></li>
<li><a href="/{{version}}/javadoc/org/apache/kafka/common/config/provider/FileConfigProvider.html"><code>FileConfigProvider</code></a></li>
</ul>
</ul>
<p>
To use a configuration provider, specify it in your configuration using the <code>config.providers</code> property.
</p>
<h4 class="anchor-heading">
<a id="using_config_providers" class="anchor-link"></a>
<a href="#using_config_providers">Using Configuration Providers</a>
</h4>
<p>Configuration providers allow you to pass parameters and retrieve configuration data from various sources.</p>
<p>To specify configuration providers, you use a comma-separated list of aliases and the fully-qualified class names that implement the configuration providers:</p>
<pre><code>config.providers=provider1,provider2
config.providers.provider1.class=com.example.Provider1
config.providers.provider2.class=com.example.Provider2</code></pre>
<p>Each provider can have its own set of parameters, which are passed in a specific format:</p>
<pre><code>config.providers.&lt;provider_alias&gt;.param.&lt;name&gt;=&lt;value&gt;</code></pre>
<p>The <code>ConfigProvider</code> interface serves as a base for all configuration providers.
Custom implementations of this interface can be created to retrieve configuration data from various sources.
You can package the implementation as a JAR file, add the JAR to your classpath, and reference the provider's class in your configuration.</p>
<p><b>Example custom provider configuration</b></p>
<pre><code>config.providers=customProvider
config.providers.customProvider.class=com.example.customProvider
config.providers.customProvider.param.param1=value1
config.providers.customProvider.param.param2=value2</code></pre>
<h4 class="anchor-heading">
<a id="directory_config_provider" class="anchor-link"></a>
<a href="#directory_config_provider">DirectoryConfigProvider</a>
</h4>
<p>The <code>DirectoryConfigProvider</code> retrieves configuration data from files stored in a specified directory.</p>
<p>Each file represents a key, and its content is the value. This provider is useful for loading multiple configuration files and for organizing configuration data into separate files.</p>
<p>To restrict the files that the <code>DirectoryConfigProvider</code> can access, use the <code>allowed.paths</code> parameter.
This parameter accepts a comma-separated list of paths that the provider is allowed to access. If not set, all paths are allowed.</p>
<p><b>Example <code>DirectoryConfigProvider</code> configuration</b></p>
<pre><code>config.providers=dirProvider
config.providers.dirProvider.class=org.apache.kafka.common.config.provider.DirectoryConfigProvider
config.providers.dirProvider.param.allowed.paths=/path/to/dir1,/path/to/dir2</code></pre>
<p>
To reference a value supplied by the <code>DirectoryConfigProvider</code>, use the correct placeholder syntax:
</p>
<pre><code>${dirProvider:&lt;path_to_file&gt;:&lt;file_name&gt;}</code></pre>
<h4 class="anchor-heading">
<a id="env_var_config_provider" class="anchor-link"></a>
<a href="#env_var_config_provider">EnvVarConfigProvider</a>
</h4>
<p>The <code>EnvVarConfigProvider</code> retrieves configuration data from environment variables.</p>
<p>No specific parameters are required, as it reads directly from the specified environment variables.</p>
<p>This provider is useful for configuring applications running in containers, for example, to load certificates or JAAS configuration from environment variables mapped from secrets.</p>
<p>To restrict which environment variables the <code>EnvVarConfigProvider</code> can access, use the <code>allowlist.pattern</code> parameter. This parameter accepts a regular expression that environment variable names must match to be used by the provider.</p>
<p><b>Example <code>EnvVarConfigProvider</code> configuration</b></p>
<pre><code>config.providers=envVarProvider
config.providers.envVarProvider.class=org.apache.kafka.common.config.provider.EnvVarConfigProvider
config.providers.envVarProvider.param.allowlist.pattern=^MY_ENVAR1_.*</code></pre>
<p>
To reference a value supplied by the <code>EnvVarConfigProvider</code>, use the correct placeholder syntax:
</p>
<pre><code>${envVarProvider:&lt;enVar_name&gt;}</code></pre>
<h4 class="anchor-heading">
<a id="file_config_provider" class="anchor-link"></a>
<a href="#file_config_provider">FileConfigProvider</a>
</h4>
<p>The <code>FileConfigProvider</code> retrieves configuration data from a single properties file.</p>
<p>This provider is useful for loading configuration data from mounted files.</p>
<p>To restrict the file paths that the <code>FileConfigProvider</code> can access, use the <code>allowed.paths</code> parameter. This parameter accepts a comma-separated list of paths that the provider is allowed to access. If not set, all paths are allowed.</p>
<p><b>Example <code>FileConfigProvider</code> configuration</b></p>
<pre><code>config.providers=fileProvider
config.providers.fileProvider.class=org.apache.kafka.common.config.provider.FileConfigProvider
config.providers.fileProvider.param.allowed.paths=/path/to/config1,/path/to/config2</code></pre>
<p>
To reference a value supplied by the <code>FileConfigProvider</code>, use the correct placeholder syntax:
</p>
<pre><code>${fileProvider:&lt;path_and_filename&gt;:&lt;property&gt;}</code></pre>
<h4 class="anchor-heading">
<a id="ref_config_provider" class="anchor-link"></a>
<a href="#ref_config_provider">Example: Referencing files</a>
</h4>
<p>
Heres an example that uses a file configuration provider with Kafka Connect to provide authentication credentials to a database for a connector.
</p>
<p>
First, create a <code>connector-credentials.properties</code> configuration file with the following credentials:
</p>
<pre><code>dbUsername=my-username
dbPassword=my-password</code></pre>
<p>
Specify a <code>FileConfigProvider</code> in the Kafka Connect configuration:
</p>
<p><b>Example Kafka Connect configuration with a <code>FileConfigProvider</code></b></p>
<pre><code>config.providers=fileProvider
config.providers.fileProvider.class=org.apache.kafka.common.config.provider.FileConfigProvider</code></pre>
<p>Next, reference the properties from the file in the connector configuration.</p>
<p><b>Example connector configuration referencing file properties</b></p>
<pre><code>database.user=${fileProvider:/path/to/connector-credentials.properties:dbUsername}
database.password=${fileProvider:/path/to/connector-credentials.properties:dbPassword}</code></pre>
<p>At runtime, the configuration provider reads and extracts the values from the properties file.</p>
</script>
<div class="p-configuration"></div>