-
Notifications
You must be signed in to change notification settings - Fork 14.8k
KAFKA-18854:Move DynamicConfig to server module #19019
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
4bb291b
dde6632
89af1ba
0256112
f7e1e1b
951353d
e08acb7
205132c
4365505
944518e
73ef6af
8955fd9
c0cdb44
245c657
d566e9b
fd90393
96f7834
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -153,6 +153,24 @@ public static void addClientSslSupport(ConfigDef config) { | |
| SslConfigs.SSL_KEYSTORE_CERTIFICATE_CHAIN_CONFIG, | ||
| SslConfigs.SSL_KEYSTORE_KEY_CONFIG, | ||
| SslConfigs.SSL_TRUSTSTORE_CERTIFICATES_CONFIG); | ||
|
|
||
| public static final Set<String> DYNAMIC_LISTENER_CONFIGS = Set.of( | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ditto |
||
| SslConfigs.SSL_PROTOCOL_CONFIG, | ||
| SslConfigs.SSL_PROVIDER_CONFIG, | ||
| SslConfigs.SSL_CIPHER_SUITES_CONFIG, | ||
| SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, | ||
| SslConfigs.SSL_KEYSTORE_TYPE_CONFIG, | ||
| SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, | ||
| SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, | ||
| SslConfigs.SSL_KEY_PASSWORD_CONFIG, | ||
| SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, | ||
| SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, | ||
| SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, | ||
| SslConfigs.SSL_KEYMANAGER_ALGORITHM_CONFIG, | ||
| SslConfigs.SSL_TRUSTMANAGER_ALGORITHM_CONFIG, | ||
| SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, | ||
| SslConfigs.SSL_SECURE_RANDOM_IMPLEMENTATION_CONFIG, | ||
| SslConfigs.SSL_ENGINE_FACTORY_CLASS_CONFIG); | ||
|
|
||
| public static final Set<String> NON_RECONFIGURABLE_CONFIGS = Set.of( | ||
| BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,31 +17,30 @@ | |
|
|
||
| package kafka.server | ||
|
|
||
| import java.util | ||
| import java.util.{Collections, Properties} | ||
| import java.util.concurrent.CopyOnWriteArrayList | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock | ||
| import kafka.log.{LogCleaner, LogManager} | ||
| import kafka.log.LogManager | ||
| import kafka.network.{DataPlaneAcceptor, SocketServer} | ||
| import kafka.server.DynamicBrokerConfig._ | ||
| import kafka.utils.{CoreUtils, Logging} | ||
| import org.apache.kafka.common.Reconfigurable | ||
| import org.apache.kafka.network.EndPoint | ||
| import org.apache.kafka.common.config.internals.BrokerSecurityConfigs | ||
| import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs} | ||
| import org.apache.kafka.common.config._ | ||
| import org.apache.kafka.common.metrics.{Metrics, MetricsReporter} | ||
| import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable} | ||
| import org.apache.kafka.common.security.authenticator.LoginManager | ||
| import org.apache.kafka.common.utils.{ConfigUtils, Utils} | ||
| import org.apache.kafka.coordinator.transaction.TransactionLogConfig | ||
| import org.apache.kafka.network.SocketServerConfigs | ||
| import org.apache.kafka.network.{EndPoint, SocketServerConfigs} | ||
| import org.apache.kafka.server.ProcessRole | ||
| import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms} | ||
| import org.apache.kafka.server.config._ | ||
| import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig | ||
| import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs} | ||
| import org.apache.kafka.server.telemetry.ClientTelemetry | ||
| import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig} | ||
|
|
||
| import java.util | ||
| import java.util.concurrent.CopyOnWriteArrayList | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock | ||
| import java.util.{Collections, Properties} | ||
| import scala.collection._ | ||
| import scala.jdk.CollectionConverters._ | ||
|
|
||
|
|
@@ -82,16 +81,6 @@ object DynamicBrokerConfig { | |
| private[server] val DynamicSecurityConfigs = SslConfigs.RECONFIGURABLE_CONFIGS.asScala | ||
| private[server] val DynamicProducerStateManagerConfig = Set(TransactionLogConfig.PRODUCER_ID_EXPIRATION_MS_CONFIG, TransactionLogConfig.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG) | ||
|
|
||
| val AllDynamicConfigs = DynamicSecurityConfigs ++ | ||
| LogCleaner.ReconfigurableConfigs ++ | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please move
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @chia7712 PTAL |
||
| DynamicLogConfig.ReconfigurableConfigs ++ | ||
| DynamicThreadPool.ReconfigurableConfigs ++ | ||
| Set(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG) ++ | ||
| DynamicListenerConfig.ReconfigurableConfigs ++ | ||
| SocketServer.ReconfigurableConfigs ++ | ||
| DynamicProducerStateManagerConfig ++ | ||
| DynamicRemoteLogConfig.ReconfigurableConfigs | ||
|
|
||
| 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( | ||
| ClusterLevelListenerConfigs) | ||
|
|
@@ -152,7 +141,7 @@ object DynamicBrokerConfig { | |
| } | ||
|
|
||
| private def nonDynamicConfigs(props: Properties): Set[String] = { | ||
| props.asScala.keySet.intersect(DynamicConfig.Broker.nonDynamicProps) | ||
| props.asScala.keySet.intersect(DynamicConfig.Broker.NON_DYNAMIC_PROPS.asScala) | ||
| } | ||
|
|
||
| private def securityConfigsWithoutListenerPrefix(props: Properties): Set[String] = { | ||
|
|
@@ -169,7 +158,7 @@ object DynamicBrokerConfig { | |
| } | ||
|
|
||
| private[server] def dynamicConfigUpdateModes: util.Map[String, String] = { | ||
| AllDynamicConfigs.map { name => | ||
| DynamicConfig.ALL_DYNAMIC_CONFIGS.asScala.map { name => | ||
| val mode = if (PerBrokerConfigs.contains(name)) "per-broker" else "cluster-wide" | ||
| name -> mode | ||
| }.toMap.asJava | ||
|
|
@@ -282,7 +271,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging | |
| } | ||
|
|
||
| private def verifyReconfigurableConfigs(configNames: Set[String]): Unit = { | ||
| val nonDynamic = configNames.intersect(DynamicConfig.Broker.nonDynamicProps) | ||
| val nonDynamic = configNames.intersect(DynamicConfig.Broker.NON_DYNAMIC_PROPS.asScala) | ||
| require(nonDynamic.isEmpty, s"Reconfigurable contains non-dynamic configs $nonDynamic") | ||
| } | ||
|
|
||
|
|
||
This file was deleted.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
SaslConfigs.javais public, so maybe we can move this set toDynamicConfig?