diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index e5e58f78761b9..03a1737490237 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -57,6 +57,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.metadata.BrokerState; import org.apache.kafka.network.SocketServerConfigs; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.storage.internals.log.CleanerConfig; import org.slf4j.Logger; @@ -163,7 +164,7 @@ public void start() { private void doStart() { brokerConfig.put(ZkConfigs.ZK_CONNECT_CONFIG, zKConnectString()); - putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, true); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, (short) brokers.length); putIfAbsent(brokerConfig, AUTO_CREATE_TOPICS_ENABLE_CONFIG, false); @@ -178,7 +179,7 @@ private void doStart() { listenerName = new ListenerName(listenerConfig.toString()); for (int i = 0; i < brokers.length; i++) { - brokerConfig.put(KafkaConfig.BrokerIdProp(), i); + brokerConfig.put(KafkaServerConfigs.BROKER_ID_CONFIG, i); currentBrokerLogDirs[i] = currentBrokerLogDirs[i] == null ? createLogDir() : currentBrokerLogDirs[i]; brokerConfig.put(LOG_DIR_CONFIG, currentBrokerLogDirs[i]); if (!hasListenerConfig) diff --git a/core/src/main/java/kafka/log/remote/RemoteLogManager.java b/core/src/main/java/kafka/log/remote/RemoteLogManager.java index 6555b7c0cda1a..377f3fae0862d 100644 --- a/core/src/main/java/kafka/log/remote/RemoteLogManager.java +++ b/core/src/main/java/kafka/log/remote/RemoteLogManager.java @@ -21,7 +21,6 @@ import kafka.cluster.Partition; import kafka.log.UnifiedLog; import kafka.server.BrokerTopicStats; -import kafka.server.KafkaConfig; import kafka.server.StopPartition; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicIdPartition; @@ -44,6 +43,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.log.remote.metadata.storage.ClassLoaderAwareRemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; @@ -252,7 +252,7 @@ public RemoteStorageManager run() { private void configureRSM() { final Map rsmProps = new HashMap<>(rlmConfig.remoteStorageManagerProps()); - rsmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rsmProps.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId); remoteLogStorageManager.configure(rsmProps); } @@ -286,7 +286,7 @@ private void configureRLMM() { // update the remoteLogMetadataProps here to override endpoint config if any rlmmProps.putAll(rlmConfig.remoteLogMetadataManagerProps()); - rlmmProps.put(KafkaConfig.BrokerIdProp(), brokerId); + rlmmProps.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId); rlmmProps.put(LOG_DIR_CONFIG, logDir); rlmmProps.put("cluster.id", clusterId); diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index c4782dd95cea3..0eb852f38e984 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -36,7 +36,7 @@ import org.apache.kafka.common.requests.TransactionResult import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation} @@ -812,7 +812,7 @@ private[transaction] case class TransactionConfig(transactionalIdExpirationMs: I transactionLogMinInsyncReplicas: Int = TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_DEFAULT, abortTimedOutTransactionsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_DEFAULT, removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_DEFAULT, - requestTimeoutMs: Int = Defaults.REQUEST_TIMEOUT_MS) + requestTimeoutMs: Int = KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT) case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) { override def toString: String = { diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index 66afce2a1e542..f2c5c05b95971 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record.MemoryRecords.RecordFilter.BatchRetention import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{BufferSupplier, Time} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, LastRecord, LogDirFailureChannel, LogSegment, LogSegmentOffsetOverflowException, OffsetMap, SkimpyOffsetMap, TransactionIndex} @@ -499,7 +500,7 @@ object LogCleaner { CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, - KafkaConfig.MessageMaxBytesProp, + KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP ) diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 6dd4bc5051328..b79cd97833619 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -47,7 +47,7 @@ import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time, Utils} import org.apache.kafka.common.{Endpoint, KafkaException, MetricName, Reconfigurable} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.FutureUtils import org.slf4j.event.Level @@ -425,7 +425,7 @@ object SocketServer { object DataPlaneAcceptor { val ThreadPrefix: String = "data-plane" val MetricPrefix: String = "" - val ListenerReconfigurableConfigs: Set[String] = Set(KafkaConfig.NumNetworkThreadsProp) + val ListenerReconfigurableConfigs: Set[String] = Set(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG) } class DataPlaneAcceptor(socketServer: SocketServer, @@ -506,7 +506,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * the configs have passed validation using [[validateReconfiguration( Map )]]. */ override def reconfigure(configs: util.Map[String, _]): Unit = { - val newNumNetworkThreads = configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int] + val newNumNetworkThreads = configs.get(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int] if (newNumNetworkThreads != processors.length) { info(s"Resizing network thread pool size for ${endPoint.listenerName} listener from ${processors.length} to $newNumNetworkThreads") @@ -522,7 +522,7 @@ class DataPlaneAcceptor(socketServer: SocketServer, * Configure this class with the given key-value pairs */ override def configure(configs: util.Map[String, _]): Unit = { - addProcessors(configs.get(KafkaConfig.NumNetworkThreadsProp).asInstanceOf[Int]) + addProcessors(configs.get(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int]) } } diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 4c320e7c6f5f8..e4c9cd5543484 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -39,7 +39,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.server.ProcessRole -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} +import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs} import org.apache.kafka.server.telemetry.ClientTelemetry @@ -98,7 +98,7 @@ object DynamicBrokerConfig { DynamicProducerStateManagerConfig ++ DynamicRemoteLogConfig.ReconfigurableConfigs - private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, KafkaConfig.NumNetworkThreadsProp) + private val ClusterLevelListenerConfigs = Set(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG) private val PerBrokerConfigs = (DynamicSecurityConfigs ++ DynamicListenerConfig.ReconfigurableConfigs).diff( ClusterLevelListenerConfigs) private val ListenerMechanismConfigs = Set(KafkaSecurityConfigs.SASL_JAAS_CONFIG, @@ -769,10 +769,10 @@ class DynamicLogConfig(logManager: LogManager, server: KafkaBroker) extends Brok object DynamicThreadPool { val ReconfigurableConfigs = Set( - KafkaConfig.NumIoThreadsProp, + KafkaServerConfigs.NUM_IO_THREADS_CONFIG, ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, - KafkaConfig.BackgroundThreadsProp) + KafkaServerConfigs.BACKGROUND_THREADS_CONFIG) def validateReconfiguration(currentConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { newConfig.values.forEach { (k, v) => @@ -794,10 +794,10 @@ object DynamicThreadPool { def getValue(config: KafkaConfig, name: String): Int = { name match { - case KafkaConfig.NumIoThreadsProp => config.numIoThreads + case KafkaServerConfigs.NUM_IO_THREADS_CONFIG => config.numIoThreads case ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG => config.numReplicaFetchers case ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG => config.numRecoveryThreadsPerDataDir - case KafkaConfig.BackgroundThreadsProp => config.backgroundThreads + case KafkaServerConfigs.BACKGROUND_THREADS_CONFIG => config.backgroundThreads case n => throw new IllegalStateException(s"Unexpected config $n") } } @@ -806,7 +806,7 @@ object DynamicThreadPool { class ControllerDynamicThreadPool(controller: ControllerServer) extends BrokerReconfigurable { override def reconfigurableConfigs: Set[String] = { - Set(KafkaConfig.NumIoThreadsProp) + Set(KafkaServerConfigs.NUM_IO_THREADS_CONFIG) } override def validateReconfiguration(newConfig: KafkaConfig): Unit = { @@ -899,7 +899,7 @@ class DynamicMetricsReporters(brokerId: Int, config: KafkaConfig, metrics: Metri class DynamicMetricReporterState(brokerId: Int, config: KafkaConfig, metrics: Metrics, clusterId: String) { private[server] val dynamicConfig = config.dynamicConfig - private val propsOverride = Map[String, AnyRef](KafkaConfig.BrokerIdProp -> brokerId.toString) + private val propsOverride = Map[String, AnyRef](KafkaServerConfigs.BROKER_ID_CONFIG -> brokerId.toString) private[server] val currentReporters = mutable.Map[String, MetricsReporter]() createReporters(config, clusterId, metricsReporterClasses(dynamicConfig.currentKafkaConfig.values()).asJava, Collections.emptyMap[String, Object]) @@ -1009,7 +1009,7 @@ object DynamicListenerConfig { SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG, // Network threads - KafkaConfig.NumNetworkThreadsProp + KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG ) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9a228e014365a..125b5c3db014a 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -24,7 +24,6 @@ import kafka.cluster.EndPoint import kafka.utils.CoreUtils.parseCsvList import kafka.utils.{CoreUtils, Logging} import kafka.utils.Implicits._ -import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.Reconfigurable import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig} @@ -48,7 +47,7 @@ import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} import org.apache.kafka.server.common.MetadataVersion._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.server.record.BrokerCompressionType @@ -93,114 +92,6 @@ object KafkaConfig { zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined } - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableProp = "broker.id.generation.enable" - val MaxReservedBrokerIdProp = "reserved.broker.max.id" - val BrokerIdProp = "broker.id" - val MessageMaxBytesProp = "message.max.bytes" - val NumNetworkThreadsProp = "num.network.threads" - val NumIoThreadsProp = "num.io.threads" - val BackgroundThreadsProp = "background.threads" - val NumReplicaAlterLogDirsThreadsProp = "num.replica.alter.log.dirs.threads" - val QueuedMaxRequestsProp = "queued.max.requests" - val QueuedMaxBytesProp = "queued.max.request.bytes" - val RequestTimeoutMsProp = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG - val ConnectionSetupTimeoutMaxMsProp = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG - val DeleteTopicEnableProp = "delete.topic.enable" - val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) - val CompressionGzipLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG) - val CompressionLz4LevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG) - val CompressionZstdLevelProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG) - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameProp = "authorizer.class.name" - val EarlyStartListenersProp = "early.start.listeners" - - /***************** rack configuration *************/ - val RackProp = "broker.rack" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesProp = "controlled.shutdown.max.retries" - val ControlledShutdownRetryBackoffMsProp = "controlled.shutdown.retry.backoff.ms" - val ControlledShutdownEnableProp = "controlled.shutdown.enable" - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlots = "max.incremental.fetch.session.cache.slots" - val FetchMaxBytes = "fetch.max.bytes" - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimit = "max.request.partition.size.limit" - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasProp = "delegation.token.master.key" - val DelegationTokenSecretKeyProp = "delegation.token.secret.key" - val DelegationTokenMaxLifeTimeProp = "delegation.token.max.lifetime.ms" - val DelegationTokenExpiryTimeMsProp = "delegation.token.expiry.time.ms" - val DelegationTokenExpiryCheckIntervalMsProp = "delegation.token.expiry.check.interval.ms" - - /** Internal Configurations **/ - val UnstableApiVersionsEnableProp = "unstable.api.versions.enable" - val UnstableMetadataVersionsEnableProp = "unstable.metadata.versions.enable" - - /* Documentation */ - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnableDoc = s"Enable automatic broker id generation on the server. When enabled the value configured for $MaxReservedBrokerIdProp should be reviewed." - val MaxReservedBrokerIdDoc = "Max number that can be used for a broker.id" - val BrokerIdDoc = "The broker id for this server. If unset, a unique broker id will be generated." + - "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + - "start from " + MaxReservedBrokerIdProp + " + 1." - val MessageMaxBytesDoc = TopicConfig.MAX_MESSAGE_BYTES_DOC + - s"This can be set per topic with the topic level ${TopicConfig.MAX_MESSAGE_BYTES_CONFIG} config." - val NumNetworkThreadsDoc = s"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." - val NumIoThreadsDoc = "The number of threads that the server uses for processing requests, which may include disk I/O" - val NumReplicaAlterLogDirsThreadsDoc = "The number of threads that can move replicas between log directories, which may include disk I/O" - val BackgroundThreadsDoc = "The number of threads to use for various background processing tasks" - val QueuedMaxRequestsDoc = "The number of queued requests allowed for data-plane, before blocking the network threads" - val QueuedMaxRequestBytesDoc = "The number of queued bytes allowed before no more requests are read" - val RequestTimeoutMsDoc = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC - val ConnectionSetupTimeoutMaxMsDoc = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC - val DeleteTopicEnableDoc = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off" - val CompressionTypeDoc = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + - "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + - "'producer' which means retain the original compression codec set by the producer." - val CompressionGzipLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'gzip'." - val CompressionLz4LevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'lz4'." - val CompressionZstdLevelDoc = s"The compression level to use if $CompressionTypeProp is set to 'zstd'." - - /************* Authorizer Configuration ***********/ - val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements ${classOf[Authorizer].getName}" + - " interface, which is used by the broker for authorization." - val EarlyStartListenersDoc = "A comma-separated list of listener names which may be started before the authorizer has finished " + - "initialization. This is useful when the authorizer is dependent on the cluster itself for bootstrapping, as is the case for " + - "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + - "will also be early start listeners. A listener should not appear in this list if it accepts external traffic." - - /************* Rack Configuration **************/ - val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1, us-east-1d" - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetriesDoc = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens" - val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." - val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server." - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlotsDoc = "The maximum number of total incremental fetch sessions that we will maintain. FetchSessionCache is sharded into 8 shards and the limit is equally divided among all shards. Sessions are allocated to each shard in round-robin. Only entries within a shard are considered eligible for eviction." - val FetchMaxBytesDoc = "The maximum number of bytes we will return for a fetch request. Must be at least 1024." - - /** ********* Request Limit Configuration **************/ - val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request." - - /** ********* Delegation Token Configuration ****************/ - val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config." - val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + - " If using Kafka with KRaft, the key must also be set across all controllers. " + - " If the key is not set or set to empty string, brokers will disable the delegation token support." - val DelegationTokenMaxLifeTimeDoc = "The token has a maximum lifetime beyond which it cannot be renewed anymore. Default value 7 days." - val DelegationTokenExpiryTimeMsDoc = "The token validity time in milliseconds before the token needs to be renewed. Default value 1 day." - val DelegationTokenExpiryCheckIntervalDoc = "Scan interval to remove expired delegation tokens." - @nowarn("cat=deprecation") val configDef = { import ConfigDef.Importance._ @@ -232,19 +123,19 @@ object KafkaConfig { .define(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG, BOOLEAN, ZkConfigs.ZK_SSL_OCSP_ENABLE, LOW, ZkConfigs.ZK_SSL_OCSP_ENABLE_DOC) /** ********* General Configuration ***********/ - .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BROKER_ID_GENERATION_ENABLE, MEDIUM, BrokerIdGenerationEnableDoc) - .define(MaxReservedBrokerIdProp, INT, Defaults.MAX_RESERVED_BROKER_ID, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) - .define(BrokerIdProp, INT, Defaults.BROKER_ID, HIGH, BrokerIdDoc) - .define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc) - .define(NumNetworkThreadsProp, INT, Defaults.NUM_NETWORK_THREADS, atLeast(1), HIGH, NumNetworkThreadsDoc) - .define(NumIoThreadsProp, INT, Defaults.NUM_IO_THREADS, atLeast(1), HIGH, NumIoThreadsDoc) - .define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc) - .define(BackgroundThreadsProp, INT, Defaults.BACKGROUND_THREADS, atLeast(1), HIGH, BackgroundThreadsDoc) - .define(QueuedMaxRequestsProp, INT, Defaults.QUEUED_MAX_REQUESTS, atLeast(1), HIGH, QueuedMaxRequestsDoc) - .define(QueuedMaxBytesProp, LONG, Defaults.QUEUED_MAX_REQUEST_BYTES, MEDIUM, QueuedMaxRequestBytesDoc) - .define(RequestTimeoutMsProp, INT, Defaults.REQUEST_TIMEOUT_MS, HIGH, RequestTimeoutMsDoc) - .define(ConnectionSetupTimeoutMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ConnectionSetupTimeoutMsDoc) - .define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ConnectionSetupTimeoutMaxMsDoc) + .define(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, BOOLEAN, KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_DEFAULT, MEDIUM, KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_DOC) + .define(KafkaServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, INT, KafkaServerConfigs.RESERVED_BROKER_MAX_ID_DEFAULT, atLeast(0), MEDIUM, KafkaServerConfigs.RESERVED_BROKER_MAX_ID_DOC) + .define(KafkaServerConfigs.BROKER_ID_CONFIG, INT, KafkaServerConfigs.BROKER_ID_DEFAULT, HIGH, KafkaServerConfigs.BROKER_ID_DOC) + .define(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, KafkaServerConfigs.MESSAGE_MAX_BYTES_DOC) + .define(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, INT, KafkaServerConfigs.NUM_NETWORK_THREADS_DEFAULT, atLeast(1), HIGH, KafkaServerConfigs.NUM_NETWORK_THREADS_DOC) + .define(KafkaServerConfigs.NUM_IO_THREADS_CONFIG, INT, KafkaServerConfigs.NUM_IO_THREADS_DEFAULT, atLeast(1), HIGH, KafkaServerConfigs.NUM_IO_THREADS_DOC) + .define(KafkaServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG, INT, null, HIGH, KafkaServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC) + .define(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, INT, KafkaServerConfigs.BACKGROUND_THREADS_DEFAULT, atLeast(1), HIGH, KafkaServerConfigs.BACKGROUND_THREADS_DOC) + .define(KafkaServerConfigs.QUEUED_MAX_REQUESTS_CONFIG, INT, KafkaServerConfigs.QUEUED_MAX_REQUESTS_DEFAULT, atLeast(1), HIGH, KafkaServerConfigs.QUEUED_MAX_REQUESTS_DOC) + .define(KafkaServerConfigs.QUEUED_MAX_BYTES_CONFIG, LONG, KafkaServerConfigs.QUEUED_MAX_REQUEST_BYTES_DEFAULT, MEDIUM, KafkaServerConfigs.QUEUED_MAX_REQUEST_BYTES_DOC) + .define(KafkaServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, INT, KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT, HIGH, KafkaServerConfigs.REQUEST_TIMEOUT_MS_DOC) + .define(KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG, LONG, KafkaServerConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC) + .define(KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG, LONG, KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC) /* * KRaft mode configs. @@ -272,8 +163,8 @@ object KafkaConfig { MEDIUM, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DOC) /************* Authorizer Configuration ***********/ - .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) - .define(EarlyStartListenersProp, STRING, null, HIGH, EarlyStartListenersDoc) + .define(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, STRING, KafkaServerConfigs.AUTHORIZER_CLASS_NAME_DEFAULT, new ConfigDef.NonNullValidator(), LOW, KafkaServerConfigs.AUTHORIZER_CLASS_NAME_DOC) + .define(KafkaServerConfigs.EARLY_START_LISTENERS_CONFIG, STRING, null, HIGH, KafkaServerConfigs.EARLY_START_LISTENERS_DOC) /** ********* Socket Server Configuration ***********/ .define(SocketServerConfigs.LISTENERS_CONFIG, STRING, SocketServerConfigs.LISTENERS_DEFAULT, HIGH, SocketServerConfigs.LISTENERS_DOC) @@ -292,7 +183,7 @@ object KafkaConfig { .define(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, INT, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DEFAULT, atLeast(0), LOW, SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_DOC) /************ Rack Configuration ******************/ - .define(RackProp, STRING, null, MEDIUM, RackDoc) + .define(KafkaServerConfigs.BROKER_RACK_CONFIG, STRING, null, MEDIUM, KafkaServerConfigs.BROKER_RACK_DOC) /** ********* Log Configuration ***********/ .define(ServerLogConfigs.NUM_PARTITIONS_CONFIG, INT, ServerLogConfigs.NUM_PARTITIONS_DEFAULT, atLeast(1), MEDIUM, ServerLogConfigs.NUM_PARTITIONS_DOC) @@ -373,9 +264,9 @@ object KafkaConfig { /** ********* Controlled shutdown configuration ***********/ - .define(ControlledShutdownMaxRetriesProp, INT, Defaults.CONTROLLED_SHUTDOWN_MAX_RETRIES, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS, MEDIUM, ControlledShutdownRetryBackoffMsDoc) - .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.CONTROLLED_SHUTDOWN_ENABLE, MEDIUM, ControlledShutdownEnableDoc) + .define(KafkaServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG, INT, KafkaServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DEFAULT, MEDIUM, KafkaServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_DOC) + .define(KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, LONG, KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DEFAULT, MEDIUM, KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DOC) + .define(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, BOOLEAN, KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DEFAULT, MEDIUM, KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_DOC) /** ********* Group coordinator configuration ***********/ .define(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DEFAULT, MEDIUM, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_DOC) @@ -412,11 +303,11 @@ object KafkaConfig { .define(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG, LONG, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG, INT, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DEFAULT, atLeast(1), HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_DOC) .define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC) - .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc) - .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) - .define(CompressionGzipLevelProp, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, CompressionGzipLevelDoc) - .define(CompressionLz4LevelProp, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, CompressionLz4LevelDoc) - .define(CompressionZstdLevelProp, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, CompressionZstdLevelDoc) + .define(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, BOOLEAN, KafkaServerConfigs.DELETE_TOPIC_ENABLE_DEFAULT, HIGH, KafkaServerConfigs.DELETE_TOPIC_ENABLE_DOC) + .define(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, KafkaServerConfigs.COMPRESSION_TYPE_DOC) + .define(KafkaServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, KafkaServerConfigs.COMPRESSION_GZIP_LEVEL_DOC) + .define(KafkaServerConfigs.COMPRESSION_LZ_4_LEVEL_CONFIG, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, KafkaServerConfigs.COMPRESSION_LZ_4_LEVEL_DOC) + .define(KafkaServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, KafkaServerConfigs.COMPRESSION_ZSTD_LEVEL_DOC) /** ********* Transaction management configuration ***********/ .define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC) @@ -436,11 +327,11 @@ object KafkaConfig { .defineInternal(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG, INT, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_DOC) /** ********* Fetch Configuration **************/ - .define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc) - .define(FetchMaxBytes, INT, Defaults.FETCH_MAX_BYTES, atLeast(1024), MEDIUM, FetchMaxBytesDoc) + .define(KafkaServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG, INT, KafkaServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DEFAULT, atLeast(0), MEDIUM, KafkaServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DOC) + .define(KafkaServerConfigs.FETCH_MAX_BYTES_CONFIG, INT, KafkaServerConfigs.FETCH_MAX_BYTES_DEFAULT, atLeast(1024), MEDIUM, KafkaServerConfigs.FETCH_MAX_BYTES_DOC) /** ********* Request Limit Configuration ***********/ - .define(MaxRequestPartitionSizeLimit, INT, Defaults.MAX_REQUEST_PARTITION_SIZE_LIMIT, atLeast(1), MEDIUM, MaxRequestPartitionSizeLimitDoc) + .define(KafkaServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG, INT, KafkaServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DEFAULT, atLeast(1), MEDIUM, KafkaServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_DOC) /** ********* Kafka Metrics Configuration ***********/ .define(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG, INT, MetricConfigs.METRIC_NUM_SAMPLES_DEFAULT, atLeast(1), LOW, MetricConfigs.METRIC_NUM_SAMPLES_DOC) @@ -532,11 +423,11 @@ object KafkaConfig { .define(KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_CONFIG, STRING, null, LOW, KafkaSecurityConfigs.SASL_OAUTHBEARER_EXPECTED_ISSUER_DOC) /** ********* Delegation Token Configuration ****************/ - .define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc) - .define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc) - .define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc) - .define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc) - .define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, PASSWORD, null, MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT, atLeast(1), MEDIUM, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DOC) + .define(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, LONG, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_DEFAULT, atLeast(1), LOW, DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_DOC) /** ********* Password encryption configuration for dynamic configs *********/ .define(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, PASSWORD, null, MEDIUM, PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_DOC) @@ -557,9 +448,9 @@ object KafkaConfig { /** Internal Configurations **/ // This indicates whether unreleased APIs should be advertised by this node. - .defineInternal(UnstableApiVersionsEnableProp, BOOLEAN, false, HIGH) + .defineInternal(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) // This indicates whether unreleased MetadataVersions should be enabled on this node. - .defineInternal(UnstableMetadataVersionsEnableProp, BOOLEAN, false, HIGH) + .defineInternal(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, BOOLEAN, false, HIGH) } /** ********* Remote Log Management Configuration *********/ @@ -636,10 +527,10 @@ object KafkaConfig { */ def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = { val output = new util.HashMap[Any, Any](input) - val brokerId = output.get(KafkaConfig.BrokerIdProp) + val brokerId = output.get(KafkaServerConfigs.BROKER_ID_CONFIG) val nodeId = output.get(KRaftConfigs.NODE_ID_CONFIG) if (brokerId == null && nodeId != null) { - output.put(KafkaConfig.BrokerIdProp, nodeId) + output.put(KafkaServerConfigs.BROKER_ID_CONFIG, nodeId) } else if (brokerId != null && nodeId == null) { output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId) } @@ -773,9 +664,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val ZkSslCrlEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG) val ZkSslOcspEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG) /** ********* General Configuration ***********/ - val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp) - val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) - var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val brokerIdGenerationEnable: Boolean = getBoolean(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG) + val maxReservedBrokerId: Int = getInt(KafkaServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG) + var brokerId: Int = getInt(KafkaServerConfigs.BROKER_ID_CONFIG) val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG) val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG) val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG) @@ -825,18 +716,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami def metadataLogSegmentMinBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG) val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG) - def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) - def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) - val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) - val queuedMaxBytes = getLong(KafkaConfig.QueuedMaxBytesProp) - def numIoThreads = getInt(KafkaConfig.NumIoThreadsProp) - def messageMaxBytes = getInt(KafkaConfig.MessageMaxBytesProp) - val requestTimeoutMs = getInt(KafkaConfig.RequestTimeoutMsProp) - val connectionSetupTimeoutMs = getLong(KafkaConfig.ConnectionSetupTimeoutMsProp) - val connectionSetupTimeoutMaxMs = getLong(KafkaConfig.ConnectionSetupTimeoutMaxMsProp) + def numNetworkThreads = getInt(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG) + def backgroundThreads = getInt(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG) + val queuedMaxRequests = getInt(KafkaServerConfigs.QUEUED_MAX_REQUESTS_CONFIG) + val queuedMaxBytes = getLong(KafkaServerConfigs.QUEUED_MAX_BYTES_CONFIG) + def numIoThreads = getInt(KafkaServerConfigs.NUM_IO_THREADS_CONFIG) + def messageMaxBytes = getInt(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG) + val requestTimeoutMs = getInt(KafkaServerConfigs.REQUEST_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMs = getLong(KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG) + val connectionSetupTimeoutMaxMs = getLong(KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG) def getNumReplicaAlterLogDirsThreads: Int = { - val numThreads: Integer = Option(getInt(KafkaConfig.NumReplicaAlterLogDirsThreadsProp)).getOrElse(logDirs.size) + val numThreads: Integer = Option(getInt(KafkaServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).getOrElse(logDirs.size) numThreads } @@ -850,7 +741,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami /************* Authorizer Configuration ***********/ def createNewAuthorizer(): Option[Authorizer] = { - val className = getString(KafkaConfig.AuthorizerClassNameProp) + val className = getString(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG) if (className == null || className.isEmpty) None else { @@ -861,13 +752,13 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val earlyStartListeners: Set[ListenerName] = { val listenersSet = listeners.map(_.listenerName).toSet val controllerListenersSet = controllerListeners.map(_.listenerName).toSet - Option(getString(KafkaConfig.EarlyStartListenersProp)) match { + Option(getString(KafkaServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { case None => controllerListenersSet case Some(str) => str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str => val listenerName = new ListenerName(str) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) - throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " + + throw new ConfigException(s"${KafkaServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " + s"listener ${listenerName.value()}, but this is not contained in " + s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}") listenerName @@ -889,7 +780,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val failedAuthenticationDelayMs = getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG) /***************** rack configuration **************/ - val rack = Option(getString(KafkaConfig.RackProp)) + val rack = Option(getString(KafkaServerConfigs.BROKER_RACK_CONFIG)) val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG)) /** ********* Log Configuration ***********/ @@ -1020,9 +911,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami } /** ********* Controlled shutdown configuration ***********/ - val controlledShutdownMaxRetries = getInt(KafkaConfig.ControlledShutdownMaxRetriesProp) - val controlledShutdownRetryBackoffMs = getLong(KafkaConfig.ControlledShutdownRetryBackoffMsProp) - val controlledShutdownEnable = getBoolean(KafkaConfig.ControlledShutdownEnableProp) + val controlledShutdownMaxRetries = getInt(KafkaServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG) + val controlledShutdownRetryBackoffMs = getLong(KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG) + val controlledShutdownEnable = getBoolean(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG) /** ********* Feature configuration ***********/ def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported @@ -1118,12 +1009,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled /** ********* DelegationToken Configuration **************/ - val delegationTokenSecretKey = Option(getPassword(KafkaConfig.DelegationTokenSecretKeyProp)) - .getOrElse(getPassword(KafkaConfig.DelegationTokenSecretKeyAliasProp)) + val delegationTokenSecretKey = Option(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG)) + .getOrElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG)) val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey.value.nonEmpty - val delegationTokenMaxLifeMs = getLong(KafkaConfig.DelegationTokenMaxLifeTimeProp) - val delegationTokenExpiryTimeMs = getLong(KafkaConfig.DelegationTokenExpiryTimeMsProp) - val delegationTokenExpiryCheckIntervalMs = getLong(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp) + val delegationTokenMaxLifeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG) + val delegationTokenExpiryTimeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG) + val delegationTokenExpiryCheckIntervalMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG) /** ********* Password encryption configuration for dynamic configs *********/ def passwordEncoderSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG)) @@ -1144,17 +1035,18 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val controllerQuotaWindowSizeSeconds = getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG) /** ********* Fetch Configuration **************/ - val maxIncrementalFetchSessionCacheSlots = getInt(KafkaConfig.MaxIncrementalFetchSessionCacheSlots) - val fetchMaxBytes = getInt(KafkaConfig.FetchMaxBytes) + val maxIncrementalFetchSessionCacheSlots = getInt(KafkaServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG) + val fetchMaxBytes = getInt(KafkaServerConfigs.FETCH_MAX_BYTES_CONFIG) /** ********* Request Limit Configuration ***********/ - val maxRequestPartitionSizeLimit = getInt(KafkaConfig.MaxRequestPartitionSizeLimit) + val maxRequestPartitionSizeLimit = getInt(KafkaServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG) + + val deleteTopicEnable = getBoolean(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG) + def compressionType = getString(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG) - val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp) - def compressionType = getString(KafkaConfig.CompressionTypeProp) - def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp) - def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp) - def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp) + def gzipCompressionLevel = getInt(KafkaServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG) + def lz4CompressionLevel = getInt(KafkaServerConfigs.COMPRESSION_LZ_4_LEVEL_CONFIG) + def zstdCompressionLevel = getInt(KafkaServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG) /** ********* Raft Quorum Configuration *********/ val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG) @@ -1166,8 +1058,8 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami val quorumRetryBackoffMs = getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG) /** Internal Configurations **/ - val unstableApiVersionsEnabled = getBoolean(KafkaConfig.UnstableApiVersionsEnableProp) - val unstableMetadataVersionsEnabled = getBoolean(KafkaConfig.UnstableMetadataVersionsEnableProp) + val unstableApiVersionsEnabled = getBoolean(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG) + val unstableMetadataVersionsEnabled = getBoolean(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG) def addReconfigurable(reconfigurable: Reconfigurable): Unit = { dynamicConfig.addReconfigurable(reconfigurable) @@ -1319,7 +1211,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami @nowarn("cat=deprecation") private def validateValues(): Unit = { if (nodeId != brokerId) { - throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${KafkaConfig.BrokerIdProp}`.") + throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${KafkaServerConfigs.BROKER_ID_CONFIG}`.") } if (requiresZookeeper) { if (zkConnect == null) { @@ -1519,7 +1411,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol), s"${KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication") require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes, - s"${KafkaConfig.QueuedMaxBytesProp} must be larger or equal to ${SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG}") + s"${KafkaServerConfigs.QUEUED_MAX_BYTES_CONFIG} must be larger or equal to ${SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG}") if (maxConnectionsPerIp == 0) require(maxConnectionsPerIpOverrides.nonEmpty, s"${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG} can be set to zero only if" + diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java index 308212ff58ddb..d672c178dd951 100644 --- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java +++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.common.OffsetAndEpoch; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.log.remote.storage.ClassLoaderAwareRemoteStorageManager; import org.apache.kafka.server.log.remote.storage.LogSegmentData; import org.apache.kafka.server.log.remote.storage.NoOpRemoteLogMetadataManager; @@ -356,7 +357,7 @@ void testRemoteLogMetadataManagerWithEndpointConfig() { assertEquals(host + ":" + port, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "bootstrap.servers")); assertEquals(securityProtocol, capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(KafkaServerConfigs.BROKER_ID_CONFIG)); } @Test @@ -395,7 +396,7 @@ public RemoteLogMetadataManager createRemoteLogMetadataManager() { // should be overridden as SSL assertEquals("SSL", capture.getValue().get(REMOTE_LOG_METADATA_COMMON_CLIENT_PREFIX + "security.protocol")); assertEquals(clusterId, capture.getValue().get("cluster.id")); - assertEquals(brokerId, capture.getValue().get(KafkaConfig.BrokerIdProp())); + assertEquals(brokerId, capture.getValue().get(KafkaServerConfigs.BROKER_ID_CONFIG)); } } diff --git a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java index 4d34ce040142b..6747dcb6c6887 100644 --- a/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java +++ b/core/src/test/java/kafka/testkit/KafkaClusterTestKit.java @@ -23,7 +23,6 @@ import kafka.server.FaultHandlerFactory; import kafka.server.SharedServer; import kafka.server.KafkaConfig; -import kafka.server.KafkaConfig$; import kafka.server.KafkaRaftServer; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -41,6 +40,7 @@ import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.config.KRaftConfigs; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.storage.internals.log.CleanerConfig; @@ -213,8 +213,8 @@ private KafkaConfig createNodeConfig(TestKitNode node) { if (controllerNode != null) { props.putAll(controllerNode.propertyOverrides()); } - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableMetadataVersionsEnableProp(), "true"); - props.putIfAbsent(KafkaConfig$.MODULE$.UnstableApiVersionsEnableProp(), "true"); + props.putIfAbsent(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true"); + props.putIfAbsent(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true"); return new KafkaConfig(props, false, Option.empty()); } diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala index 79729bae5147a..7ae0176913891 100644 --- a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.message.ApiMessageType import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.network.SocketServerConfigs +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotNull, assertTrue} import org.junit.jupiter.api.Timeout import org.junit.jupiter.params.ParameterizedTest @@ -43,7 +44,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { TestUtils.createBrokerConfigs(1, null).map(props => { // Enable unstable api versions to be compatible with the new APIs under development, // maybe we can remove this after the new APIs is complete. - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } else { @@ -54,7 +55,7 @@ class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT") props.setProperty("listeners", "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://localhost:0,CONTROLLER://localhost:0") - props.setProperty(KafkaConfig.UnstableApiVersionsEnableProp, "true") + props.setProperty(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") props }).map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala index e5c2d08e0d1ca..029d20b4db9bf 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractAuthorizerIntegrationTest.scala @@ -13,7 +13,7 @@ package kafka.api import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerConfig @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.metadata.authorizer.StandardAuthorizer +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.{BeforeEach, TestInfo} import java.util.Properties @@ -93,7 +94,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, group) override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } @@ -105,10 +106,10 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") @@ -116,7 +117,7 @@ class AbstractAuthorizerIntegrationTest extends BaseRequestTest { properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, "1") - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[PrincipalBuilder].getName) } diff --git a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala index b08b25a8adfdd..a0692c8e9e2d1 100644 --- a/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/AbstractConsumerTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.TopicPartition import kafka.utils.TestUtils -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -33,6 +33,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.common.errors.WakeupException import org.apache.kafka.coordinator.group.GroupCoordinatorConfig +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.util.ShutdownableThread import scala.collection.mutable @@ -66,7 +67,7 @@ abstract class AbstractConsumerTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { - properties.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + properties.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "3") // don't want to lose offset properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") // set small enough session timeout diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 30b0b55f349ca..5abedc68d559b 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.config.{ConfigResource, TopicConfig} import org.apache.kafka.common.errors.{InvalidConfigurationException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.utils.Utils import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, ServerLogConfigs} import org.apache.kafka.server.policy.AlterConfigPolicy import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} @@ -127,10 +127,10 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with // Set a mutable broker config val brokerResource = new ConfigResource(ConfigResource.Type.BROKER, brokers.head.config.brokerId.toString) - val brokerConfigs = Seq(new ConfigEntry(KafkaConfig.MessageMaxBytesProp, "50000")).asJava + val brokerConfigs = Seq(new ConfigEntry(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "50000")).asJava val alterResult1 = client.alterConfigs(Map(brokerResource -> new Config(brokerConfigs)).asJava) alterResult1.all.get - assertEquals(Set(KafkaConfig.MessageMaxBytesProp), validationsForResource(brokerResource).head.configs().keySet().asScala) + assertEquals(Set(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG), validationsForResource(brokerResource).head.configs().keySet().asScala) validations.clear() val topicConfigEntries1 = Seq( diff --git a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala index bcfa7f2aa6c24..83d6d33e9c748 100644 --- a/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseAdminIntegrationTest.scala @@ -19,7 +19,6 @@ package kafka.api import java.util import java.util.Properties import java.util.concurrent.ExecutionException -import kafka.server.KafkaConfig import kafka.utils.Logging import kafka.utils.TestUtils._ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateTopicsOptions, CreateTopicsResult, DescribeClusterOptions, DescribeTopicsOptions, NewTopic, TopicDescription} @@ -30,8 +29,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} @@ -204,10 +202,10 @@ abstract class BaseAdminIntegrationTest extends IntegrationTestHarness with Logg }) } configs.foreach { config => - config.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + config.setProperty(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") config.setProperty(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") config.setProperty(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, "false") - config.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + config.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // We set this in order to test that we don't expose sensitive data via describe configs. This will already be // set for subclasses with security enabled and we don't want to overwrite it. if (!config.containsKey(KafkaSecurityConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 53429f4183e1e..c1ed59784ac13 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import com.yammer.metrics.core.{Histogram, Meter} import kafka.api.QuotaTestClients._ -import kafka.server.{ClientQuotaManager, KafkaBroker, KafkaConfig, QuotaType} +import kafka.server.{ClientQuotaManager, KafkaBroker, QuotaType} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} @@ -33,7 +33,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} @@ -51,7 +51,7 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { protected def consumerClientId = "QuotasTestConsumer-1" protected def createQuotaTestClients(topic: String, leaderNode: KafkaBroker): QuotaTestClients - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "2") this.serverConfig.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") this.serverConfig.setProperty(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") diff --git a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala index d01476f999df1..c39a30cf54710 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerTopicCreationTest.scala @@ -21,14 +21,11 @@ import java.lang.{Boolean => JBoolean} import java.time.Duration import java.util import java.util.Collections - -import kafka.server.KafkaConfig import kafka.utils.{EmptyTestInfo, TestUtils} import org.apache.kafka.clients.admin.NewTopic import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} -import org.apache.kafka.server.config.ServerLogConfigs - +import org.apache.kafka.server.config.{KafkaServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} @@ -57,7 +54,7 @@ object ConsumerTopicCreationTest { private val consumerClientId = "ConsumerTestConsumer" // configure server properties - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown + this.serverConfig.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") // speed up shutdown this.serverConfig.setProperty(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, brokerAutoTopicCreationEnable.toString) // configure client properties diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 9818b6dbd554d..8cc878cb5c91d 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.{Cluster, Reconfigurable} import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth._ -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs} import org.apache.kafka.server.quota._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -65,7 +65,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { this.serverConfig.setProperty(QuotaConfigs.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, classOf[GroupedUserQuotaCallback].getName) this.serverConfig.setProperty(s"${listenerName.configPrefix}${KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}", classOf[GroupedUserPrincipalBuilder].getName) - this.serverConfig.setProperty(KafkaConfig.DeleteTopicEnableProp, "true") + this.serverConfig.setProperty(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") super.setUp(testInfo) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, @@ -367,7 +367,7 @@ class GroupedUserQuotaCallback extends ClientQuotaCallback with Reconfigurable w val partitionRatio = new ConcurrentHashMap[String, Double]() override def configure(configs: util.Map[String, _]): Unit = { - brokerId = configs.get(KafkaConfig.BrokerIdProp).toString.toInt + brokerId = configs.get(KafkaServerConfigs.BROKER_ID_CONFIG).toString.toInt callbackInstances.incrementAndGet } diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index a36f54abdc682..95cfc201cd4bd 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -17,8 +17,6 @@ package kafka.api import java.util.Properties - -import kafka.server.KafkaConfig import kafka.utils._ import kafka.tools.StorageTool import kafka.zk.ConfigEntityChangeNotificationZNode @@ -35,6 +33,7 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo} import scala.jdk.CollectionConverters._ import scala.collection.mutable.ArrayBuffer import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.config.DelegationTokenManagerConfigs class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @@ -54,8 +53,8 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest protected val privilegedAdminClientConfig = new Properties() - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") def createDelegationTokenOptions(): CreateDelegationTokenOptions = new CreateDelegationTokenOptions() diff --git a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala index 0be5e306e2948..9f5f7d1508edc 100644 --- a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala +++ b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala @@ -14,9 +14,7 @@ package kafka.api import java.util import java.util.Properties - import kafka.security.authorizer.AclAuthorizer -import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, JaasTestUtils, TestUtils} import org.apache.kafka.clients.admin._ import org.apache.kafka.common.acl.AclOperation.{ALL, ALTER, CLUSTER_ACTION, DELETE, DESCRIBE} @@ -27,7 +25,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.Utils import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNull} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -78,7 +76,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS override val brokerCount = 1 this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) var client: Admin = _ diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 60dee252420fb..c3bad3d83e9c8 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -23,7 +23,6 @@ import java.util.{Collections, Properties} import java.util.concurrent.ExecutionException import kafka.security.authorizer.AclAuthorizer import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import kafka.server._ import kafka.utils._ import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerRecords} @@ -39,8 +38,7 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth._ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} @@ -153,13 +151,13 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas if (TestInfoUtils.isKRaft(testInfo)) { this.serverConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString) this.controllerConfig.setProperty(StandardAuthorizer.SUPER_USERS_CONFIG, kafkaPrincipal.toString + ";" + "User:ANONYMOUS") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) - this.controllerConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) + this.controllerConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { // The next two configuration parameters enable ZooKeeper secure ACLs // and sets the Kafka authorizer, both necessary to enable security. this.serverConfig.setProperty(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG, "true") - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizerClass.getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, authorizerClass.getName) // Set the specific principal that can update ACLs. this.serverConfig.setProperty(AclAuthorizer.SuperUsersProp, kafkaPrincipal.toString) diff --git a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala index 6ffd0f7fb19bb..6e23ac7fc1139 100644 --- a/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupAuthorizerIntegrationTest.scala @@ -16,7 +16,7 @@ import java.util.Properties import java.util.concurrent.ExecutionException import kafka.api.GroupAuthorizerIntegrationTest._ import kafka.security.authorizer.AclAuthorizer -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.ProducerRecord @@ -32,6 +32,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -76,16 +77,16 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest { } override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.BrokerIdProp, brokerId.toString) + properties.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId.toString) addNodeProperties(properties) } private def addNodeProperties(properties: Properties): Unit = { if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) properties.put(StandardAuthorizer.SUPER_USERS_CONFIG, BrokerPrincipal.toString) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) } properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 2133019a4a871..9ab73655a566f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -46,7 +46,7 @@ import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEX import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.test.TestUtils.DEFAULT_MAX_WAIT_MS import org.junit.jupiter.api.Assertions._ @@ -426,7 +426,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { }) assertEquals(brokers(1).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource1).entries.size) - assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(1).config.brokerId.toString, configs.get(brokerResource1).get(KafkaServerConfigs.BROKER_ID_CONFIG).value) val listenerSecurityProtocolMap = configs.get(brokerResource1).get(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG) assertEquals(brokers(1).config.getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG), listenerSecurityProtocolMap.value) assertEquals(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, listenerSecurityProtocolMap.name) @@ -439,16 +439,16 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { assertFalse(truststorePassword.isDefault) assertTrue(truststorePassword.isSensitive) assertFalse(truststorePassword.isReadOnly) - val compressionType = configs.get(brokerResource1).get(KafkaConfig.CompressionTypeProp) + val compressionType = configs.get(brokerResource1).get(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG) assertEquals(brokers(1).config.compressionType, compressionType.value) - assertEquals(KafkaConfig.CompressionTypeProp, compressionType.name) + assertEquals(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, compressionType.name) assertTrue(compressionType.isDefault) assertFalse(compressionType.isSensitive) assertFalse(compressionType.isReadOnly) assertEquals(brokers(2).config.nonInternalValues.size + numInternalConfigsSet, configs.get(brokerResource2).entries.size) - assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaConfig.BrokerIdProp).value) + assertEquals(brokers(2).config.brokerId.toString, configs.get(brokerResource2).get(KafkaServerConfigs.BROKER_ID_CONFIG).value) assertEquals(brokers(2).config.logCleanerThreads.toString, configs.get(brokerResource2).get(CleanerConfig.LOG_CLEANER_THREADS_PROP).value) @@ -2735,7 +2735,7 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG).value) // Alter configs with validateOnly = true: first and third are invalid, second is valid topicConfigEntries2 = Seq(new ConfigEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, "gzip")).asJava @@ -2764,6 +2764,6 @@ object PlaintextAdminIntegrationTest { assertEquals("snappy", configs.get(topicResource2).get(TopicConfig.COMPRESSION_TYPE_CONFIG).value) - assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaConfig.CompressionTypeProp).value) + assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG).value) } } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 71cd632c0d8f0..1f3349024a283 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.{DefaultRecord, DefaultRecordBatch} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -44,7 +44,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + overridingProps.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, replicaFetchMaxPartitionBytes.toString) overridingProps.put(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_DOC, replicaFetchMaxResponseBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) diff --git a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala index b15e9f9f2d501..75b449ac2b020 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerIdExpirationTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} @@ -233,7 +233,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 47e8de08031b4..620884ffa5e27 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PRINCIPAL_STRING} import org.apache.kafka.server.authorizer.Authorizer -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ZkConfigs} import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -53,7 +53,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu override protected lazy val trustStoreFile = Some(TestUtils.tempFile("truststore", ".jks")) override def generateConfigs: Seq[KafkaConfig] = { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, authorizationAdmin.authorizerClassName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, authorizationAdmin.authorizerClassName) super.generateConfigs } diff --git a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala index fa13715b87c20..255d2f736b44a 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsBounceTest.scala @@ -27,7 +27,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.util.ShutdownableThread -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -46,10 +46,10 @@ class TransactionsBounceTest extends IntegrationTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, false.toString) - overridingProps.put(KafkaConfig.MessageMaxBytesProp, serverMessageMaxBytes.toString) + overridingProps.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, serverMessageMaxBytes.toString) // Set a smaller value for the number of partitions for the offset commit topic (__consumer_offset topic) // so that the creation of that topic/partition(s) and subsequent leader assignment doesn't take relatively long - overridingProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + overridingProps.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) overridingProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) overridingProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) overridingProps.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, 1.toString) diff --git a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala index ee39a764f2ece..c4450033a44d5 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsExpirationTest.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidPidMappingException, TransactionalIdNotFoundException} import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -208,7 +208,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala index 993e31dc5b0fd..9fedc8e05f228 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsTest.scala @@ -17,7 +17,6 @@ package kafka.api -import kafka.server.KafkaConfig import kafka.utils.TestUtils import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue} import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerGroupMetadata, ConsumerRecord, OffsetAndMetadata} @@ -26,7 +25,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.{InvalidProducerEpochException, ProducerFencedException, TimeoutException} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -67,7 +66,7 @@ class TransactionsTest extends IntegrationTestHarness { props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 3.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 2.toString) props.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 2.toString) - props.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + props.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) props.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala index 4884fc2dc7b56..6cd384ebe188e 100644 --- a/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala +++ b/core/src/test/scala/integration/kafka/api/TransactionsWithMaxInFlightOneTest.scala @@ -26,7 +26,7 @@ import org.apache.kafka.clients.consumer.Consumer import org.apache.kafka.clients.producer.KafkaProducer import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs} -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -110,7 +110,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness { serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, 1.toString) serverProps.put(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, 1.toString) - serverProps.put(KafkaConfig.ControlledShutdownEnableProp, true.toString) + serverProps.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true.toString) serverProps.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, false.toString) serverProps.put(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG, false.toString) serverProps.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index c42fbd6330e7b..a8311969376be 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -17,13 +17,13 @@ */ package kafka.network -import kafka.server.{BaseRequestTest, KafkaConfig} +import kafka.server.BaseRequestTest import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.network.SocketServerConfigs -import org.apache.kafka.server.config.Defaults +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -43,8 +43,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.put(SocketServerConfigs.LISTENERS_CONFIG, s"$internal://localhost:0, $external://localhost:0") properties.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"$internal:PLAINTEXT, $external:PLAINTEXT") - properties.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", "2") - properties.put(KafkaConfig.NumNetworkThreadsProp, Defaults.NUM_NETWORK_THREADS.toString) + properties.put(s"listener.name.${internal.toLowerCase}.${KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG}", "2") + properties.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, KafkaServerConfigs.NUM_NETWORK_THREADS_DEFAULT.toString) } @BeforeEach @@ -53,7 +53,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) assertEquals(2, getNumNetworkThreads(internal)) TestUtils.createTopicWithAdmin(admin, "test", brokers, controllerServers) - assertEquals(Defaults.NUM_NETWORK_THREADS, getNumNetworkThreads(external)) + assertEquals(KafkaServerConfigs.NUM_NETWORK_THREADS_DEFAULT, getNumNetworkThreads(external)) } @AfterEach override def tearDown(): Unit = { @@ -71,10 +71,10 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testDynamicNumNetworkThreads(quorum: String): Unit = { // Increase the base network thread count - val newBaseNetworkThreadsCount = Defaults.NUM_NETWORK_THREADS + 1 + val newBaseNetworkThreadsCount = KafkaServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1 var props = new Properties - props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString) - reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)) + props.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString) + reconfigureServers(props, (KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, newBaseNetworkThreadsCount.toString)) // Only the external listener is changed assertEquals(2, getNumNetworkThreads(internal)) @@ -83,8 +83,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { // Increase the network thread count for internal val newInternalNetworkThreadsCount = 3 props = new Properties - props.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString) - reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", newInternalNetworkThreadsCount.toString)) + props.put(s"listener.name.${internal.toLowerCase}.${KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString) + reconfigureServers(props, (s"listener.name.${internal.toLowerCase}.${KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG}", newInternalNetworkThreadsCount.toString)) // The internal listener is changed assertEquals(newInternalNetworkThreadsCount, getNumNetworkThreads(internal)) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 379b454bc84f5..12fcd182ab248 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -62,7 +62,7 @@ import org.apache.kafka.common.serialization.{StringDeserializer, StringSerializ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.{PasswordEncoder, PasswordEncoderConfigs} -import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ConfigType, KafkaSecurityConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.ShutdownableThread @@ -542,7 +542,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "20000000") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, "0.8") props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, "300000") - props.put(KafkaConfig.MessageMaxBytesProp, "40000") + props.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "40000") props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, "50000000") props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, "6000") @@ -634,7 +634,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG, "60000") props.put(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG, "10000000") props.put(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG, TimeUnit.DAYS.toMillis(1).toString) - props.put(KafkaConfig.MessageMaxBytesProp, "100000") + props.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "100000") props.put(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG, "10000") props.put(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP, TimeUnit.DAYS.toMillis(1).toString) props.put(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP, "60000") @@ -643,7 +643,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup props.put(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG, "delete") props.put(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, "false") props.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2") - props.put(KafkaConfig.CompressionTypeProp, "gzip") + props.put(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") props.put(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG, true.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG, TimestampType.LOG_APPEND_TIME.toString) props.put(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, "1000") @@ -869,15 +869,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup } val config = servers.head.config - verifyThreadPoolResize(KafkaConfig.NumIoThreadsProp, config.numIoThreads, + verifyThreadPoolResize(KafkaServerConfigs.NUM_IO_THREADS_CONFIG, config.numIoThreads, requestHandlerPrefix, mayReceiveDuplicates = false) verifyThreadPoolResize(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, config.numReplicaFetchers, fetcherThreadPrefix, mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.BackgroundThreadsProp, config.backgroundThreads, + verifyThreadPoolResize(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, config.backgroundThreads, "kafka-scheduler-", mayReceiveDuplicates = false) verifyThreadPoolResize(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, config.numRecoveryThreadsPerDataDir, "", mayReceiveDuplicates = false) - verifyThreadPoolResize(KafkaConfig.NumNetworkThreadsProp, config.numNetworkThreads, + verifyThreadPoolResize(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, config.numNetworkThreads, networkThreadPrefix, mayReceiveDuplicates = true) verifyThreads("data-plane-kafka-socket-acceptor-", config.listeners.size) @@ -1965,7 +1965,7 @@ class TestMetricsReporter extends MetricsReporter with Reconfigurable with Close } override def configure(configs: util.Map[String, _]): Unit = { - configuredBrokers += configs.get(KafkaConfig.BrokerIdProp).toString.toInt + configuredBrokers += configs.get(KafkaServerConfigs.BROKER_ID_CONFIG).toString.toInt configureCount += 1 pollingInterval = configs.get(PollingIntervalProp).toString.toInt } diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index abc92c8c9267f..34aeeb2141610 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -38,7 +38,7 @@ import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsem import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, KafkaServerConfigs} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} @@ -324,7 +324,7 @@ abstract class QuorumTestHarness extends Logging { props.putAll(overridingProps) props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller") - props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.setProperty(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true") if (props.getProperty(KRaftConfigs.NODE_ID_CONFIG) == null) { props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1000") } diff --git a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala index 3327dc4073826..1f9d60b3400c3 100644 --- a/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AclCommandTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} import org.apache.kafka.common.utils.{AppInfoParser, SecurityUtils} import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.authorizer.Authorizer +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.log4j.Level import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -110,7 +111,7 @@ class AclCommandTest extends QuorumTestHarness with Logging { super.setUp(testInfo) brokerProps = TestUtils.createBrokerConfig(0, zkConnect) - brokerProps.put(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getName) + brokerProps.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getName) brokerProps.put(AclAuthorizer.SuperUsersProp, "User:ANONYMOUS") zkArgs = Array("--authorizer-properties", "zookeeper.connect=" + zkConnect) diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala index bb2e76e8b2d34..4f249ad63d5e9 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.{TopicDeletionDisabledException, UnknownTo import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.params.ParameterizedTest @@ -489,7 +490,7 @@ class DeleteTopicTest extends QuorumTestHarness { if (isKRaftTest()) { // Restart KRaft quorum with the updated config val overridingProps = new Properties() - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, false.toString) + overridingProps.put(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, false.toString) if (implementation != null) implementation.shutdown() implementation = newKRaftQuorum(overridingProps) diff --git a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala index a363ef7f5f9bf..4d8e88b529ca2 100644 --- a/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala +++ b/core/src/test/scala/unit/kafka/controller/ControllerChannelManagerTest.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_10_2_IV0, IBP_0_9_0, IBP_1_0_IV0, IBP_2_2_IV0, IBP_2_4_IV0, IBP_2_4_IV1, IBP_2_6_IV0, IBP_2_8_IV1, IBP_3_2_IV0, IBP_3_4_IV0} -import org.apache.kafka.server.config.ZkConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -895,7 +895,7 @@ class ControllerChannelManagerTest { private def createConfig(interBrokerVersion: MetadataVersion): KafkaConfig = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, controllerId.toString) + props.put(KafkaServerConfigs.BROKER_ID_CONFIG, controllerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "zkConnect") TestUtils.setIbpAndMessageFormatVersions(props, interBrokerVersion) KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala index b46abdc5cfd1c..632f7e4f3ce9a 100644 --- a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala @@ -24,7 +24,7 @@ import kafka.utils.{Logging, TestUtils} import scala.jdk.CollectionConverters._ import org.junit.jupiter.api.{BeforeEach, TestInfo} import com.yammer.metrics.core.Gauge -import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -39,7 +39,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with private val createDeleteIterations = 3 private val overridingProps = new Properties - overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true") + overridingProps.put(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "true") overridingProps.put(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, "false") // speed up the test for UnderReplicatedPartitions, which relies on the ISR expiry thread to execute concurrently with topic creation // But the replica.lag.time.max.ms value still need to consider the slow Jenkins testing environment diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala index 876082dd074ac..9be0ba7a5efa2 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.compress.Compression import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.junit.jupiter.api.Assertions._ @@ -259,7 +260,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, cleanerConfig.dedupeBufferSize.toString) props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP, cleanerConfig.dedupeBufferLoadFactor.toString) props.put(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP, cleanerConfig.ioBufferSize.toString) - props.put(KafkaConfig.MessageMaxBytesProp, cleanerConfig.maxMessageSize.toString) + props.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, cleanerConfig.maxMessageSize.toString) props.put(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP, cleanerConfig.backoffMs.toString) props.put(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP, cleanerConfig.maxIoBytesPerSecond.toString) KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index ac8307bdf749c..32e0f9ca3d952 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -38,7 +38,7 @@ import org.apache.kafka.common.utils._ import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.CredentialProvider import org.apache.kafka.server.common.{Features, MetadataVersion} -import org.apache.kafka.server.config.QuotaConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, QuotaConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils} import org.apache.log4j.Level @@ -2051,7 +2051,7 @@ class SocketServerTest { val sslProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), trustStoreFile = Some(trustStoreFile)) sslProps.put(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:0") - sslProps.put(KafkaConfig.NumNetworkThreadsProp, "1") + sslProps.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, "1") sslProps } diff --git a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala index e6153bd6c4c3a..df3950ba69399 100644 --- a/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala +++ b/core/src/test/scala/unit/kafka/raft/RaftManagerTest.scala @@ -31,7 +31,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.ProcessRole import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -64,7 +64,7 @@ class RaftManagerTest { } props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") - props.setProperty(KafkaConfig.BrokerIdProp, nodeId.toString) + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, nodeId.toString) new KafkaConfig(props) } diff --git a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala index f482d221763f3..c75ad5c66d152 100644 --- a/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala +++ b/core/src/test/scala/unit/kafka/security/token/delegation/DelegationTokenManagerTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.network.Session import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.apache.kafka.security.authorizer.AuthorizerUtils import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KafkaSecurityConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -59,8 +59,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness { val tokenManagers = mutable.Buffer[DelegationTokenManager]() val secretKey = "secretKey" - val maxLifeTimeMsDefault = Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS - val renewTimeMsDefault = Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS + val maxLifeTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT + val renewTimeMsDefault = DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT var tokenCache: DelegationTokenCache = _ var props: Properties = _ @@ -73,7 +73,7 @@ class DelegationTokenManagerTest extends QuorumTestHarness { super.setUp(testInfo) props = TestUtils.createBrokerConfig(0, zkConnect, enableToken = true) props.put(KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG, ScramMechanism.mechanismNames().asScala.mkString(",")) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, secretKey) + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, secretKey) tokenCache = new DelegationTokenCache(ScramMechanism.mechanismNames()) } diff --git a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala index e35561ef63990..89f27bb737f16 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractMetadataRequestTest.scala @@ -24,7 +24,7 @@ import org.apache.kafka.common.message.MetadataRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.assertEquals abstract class AbstractMetadataRequestTest extends BaseRequestTest { @@ -32,7 +32,7 @@ abstract class AbstractMetadataRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(KafkaServerConfigs.BROKER_ID_CONFIG)}") } protected def requestData(topics: List[String], allowAutoTopicCreation: Boolean): MetadataRequestData = { diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala index 4449857c7b20d..f3091c8171295 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala @@ -22,7 +22,7 @@ import org.apache.kafka.common.message.AlterUserScramCredentialsRequestData import org.apache.kafka.common.message.AlterUserScramCredentialsResponseData.AlterUserScramCredentialsResult import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse} -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -36,8 +36,8 @@ import scala.jdk.CollectionConverters._ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + properties.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index b84f6761f5968..4430ef0213134 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -34,8 +34,8 @@ import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrinci import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.apache.kafka.server.common.MetadataVersion -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -55,16 +55,16 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) if (testInfo.getDisplayName.contains("quorum=kraft-IBP_3_4")) { testMetadataVersion = MetadataVersion.IBP_3_4_IV0 } } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala index 9ee92469e53f9..f0af8c2061e51 100644 --- a/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AutoTopicCreationManagerTest.scala @@ -38,6 +38,7 @@ import org.apache.kafka.common.security.auth.{KafkaPrincipal, KafkaPrincipalSerd import org.apache.kafka.common.utils.{SecurityUtils, Utils} import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.{ControllerRequestCompletionHandler, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test} @@ -65,7 +66,7 @@ class AutoTopicCreationManagerTest { @BeforeEach def setup(): Unit = { val props = TestUtils.createBrokerConfig(1, "localhost") - props.setProperty(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + props.setProperty(KafkaServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) props.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) props.setProperty(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG, internalTopicPartitions.toString) diff --git a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala index f17b27e1ee54a..00bfc66223aef 100644 --- a/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseFetchRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.message.FetchResponseData import org.apache.kafka.common.record.Record import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.AfterEach import java.util @@ -36,7 +37,7 @@ class BaseFetchRequestTest extends BaseRequestTest { protected var producer: KafkaProducer[String, String] = _ override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.FetchMaxBytes, Int.MaxValue.toString) + properties.put(KafkaServerConfigs.FETCH_MAX_BYTES_CONFIG, Int.MaxValue.toString) } @AfterEach diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index eddeeb4253d5e..63b0999151849 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, RequestHeader, ResponseHeader} import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.BrokerState +import org.apache.kafka.server.config.KafkaServerConfigs import java.io.{DataInputStream, DataOutputStream} import java.net.Socket @@ -43,7 +44,7 @@ abstract class BaseRequestTest extends IntegrationTestHarness { override def modifyConfigs(props: Seq[Properties]): Unit = { props.foreach { p => - p.put(KafkaConfig.ControlledShutdownEnableProp, "false") + p.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") brokerPropertyOverrides(p) } } diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 84a58278a42f3..73244822359ea 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -54,7 +54,7 @@ import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock} -import org.apache.kafka.server.config.KRaftConfigs +import org.apache.kafka.server.config.{KRaftConfigs, KafkaServerConfigs} import org.apache.kafka.server.util.FutureUtils import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ @@ -890,7 +890,7 @@ class ControllerApisTest { val controller = new MockController.Builder(). newInitialTopic("foo", fooId).build() val props = new Properties() - props.put(KafkaConfig.DeleteTopicEnableProp, "false") + props.put(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") controllerApis = createControllerApis(None, controller, props) val request = new DeleteTopicsRequestData() request.topics().add(new DeleteTopicState().setName("foo").setTopicId(ZERO_UUID)) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index f4b7f6bcd8d5d..08a14035978b2 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -40,7 +40,7 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse import org.apache.kafka.common.security.auth.AuthenticationContext import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.assertEquals @@ -95,7 +95,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest { override def brokerCount: Int = 1 override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index 1ef6e37720204..6081a0fe3d0c8 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.errors.InvalidPrincipalTypeException import org.apache.kafka.common.errors.DelegationTokenNotFoundException import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.SecurityUtils +import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -43,11 +44,11 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup override def brokerCount = 1 - this.serverConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "testKey") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "testKey") // Remove expired tokens every minute. - this.serverConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") - this.controllerConfig.setProperty(KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp, "5000") + this.serverConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") + this.controllerConfig.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG, "5000") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala index e377256ccc67b..857c753e9f967 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala @@ -18,12 +18,12 @@ package kafka.server import java.util.Collections - import kafka.utils._ import org.apache.kafka.common.message.DeleteTopicsRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse} +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -34,7 +34,7 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { override def kraftControllerConfigs() = { val props = super.kraftControllerConfigs() - props.head.setProperty(KafkaConfig.DeleteTopicEnableProp, "false") + props.head.setProperty(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, "false") props } diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala index eb6acd8d73e57..11085cba33cf5 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala @@ -25,7 +25,7 @@ import org.apache.kafka.common.resource.ResourceType import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -40,7 +40,7 @@ class DescribeClusterRequestTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(KafkaServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala index e8a84048b1341..2dddb991e24d5 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala @@ -21,7 +21,7 @@ import org.apache.kafka.common.message.DescribeUserScramCredentialsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, DescribeUserScramCredentialsResponse} import org.apache.kafka.metadata.authorizer.StandardAuthorizer -import org.apache.kafka.server.config.KafkaSecurityConfigs +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -33,11 +33,11 @@ import java.util.Properties */ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest { override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[DescribeCredentialsTest.TestAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[DescribeCredentialsTest.TestAuthorizer].getName) } properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala index a70cb9cfeeb9e..208ba06aa65c0 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala @@ -28,8 +28,8 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De import org.apache.kafka.common.security.auth.{AuthenticationContext, KafkaPrincipal} import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.KafkaSecurityConfigs -import org.junit.jupiter.api.{Test, BeforeEach, TestInfo} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -45,13 +45,13 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { if (TestInfoUtils.isKRaft(testInfo)) { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[StandardAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[StandardAuthorizer].getName) } else { - this.serverConfig.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AlterCredentialsTest.TestAuthorizer].getName) + this.serverConfig.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AlterCredentialsTest.TestAuthorizer].getName) } this.serverConfig.setProperty(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[AlterCredentialsTest.TestPrincipalBuilderReturningAuthorized].getName) - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") + this.serverConfig.setProperty(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index 945d17b69c0a7..2e8c128086d5b 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -37,7 +37,7 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.authorizer._ -import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, KafkaServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.{KafkaYammerMetrics, MetricConfigs} import org.apache.kafka.server.util.KafkaScheduler @@ -134,11 +134,11 @@ class DynamicBrokerConfigTest { @Test def testUpdateDynamicThreadPool(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - origProps.put(KafkaConfig.NumIoThreadsProp, "4") - origProps.put(KafkaConfig.NumNetworkThreadsProp, "2") + origProps.put(KafkaServerConfigs.NUM_IO_THREADS_CONFIG, "4") + origProps.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") origProps.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "1") origProps.put(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG, "1") - origProps.put(KafkaConfig.BackgroundThreadsProp, "3") + origProps.put(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, "3") val config = KafkaConfig(origProps) val serverMock = Mockito.mock(classOf[KafkaBroker]) @@ -165,18 +165,18 @@ class DynamicBrokerConfigTest { val props = new Properties() - props.put(KafkaConfig.NumIoThreadsProp, "8") + props.put(KafkaServerConfigs.NUM_IO_THREADS_CONFIG, "8") config.dynamicConfig.updateDefaultConfig(props) assertEquals(8, config.numIoThreads) Mockito.verify(handlerPoolMock).resizeThreadPool(newSize = 8) - props.put(KafkaConfig.NumNetworkThreadsProp, "4") + props.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, "4") config.dynamicConfig.updateDefaultConfig(props) assertEquals(4, config.numNetworkThreads) val captor: ArgumentCaptor[JMap[String, String]] = ArgumentCaptor.forClass(classOf[JMap[String, String]]) Mockito.verify(acceptorMock).reconfigure(captor.capture()) - assertTrue(captor.getValue.containsKey(KafkaConfig.NumNetworkThreadsProp)) - assertEquals(4, captor.getValue.get(KafkaConfig.NumNetworkThreadsProp)) + assertTrue(captor.getValue.containsKey(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG)) + assertEquals(4, captor.getValue.get(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG)) props.put(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG, "2") config.dynamicConfig.updateDefaultConfig(props) @@ -188,7 +188,7 @@ class DynamicBrokerConfigTest { assertEquals(2, config.numRecoveryThreadsPerDataDir) Mockito.verify(logManagerMock).resizeRecoveryThreadPool(newSize = 2) - props.put(KafkaConfig.BackgroundThreadsProp, "6") + props.put(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, "6") config.dynamicConfig.updateDefaultConfig(props) assertEquals(6, config.backgroundThreads) Mockito.verify(schedulerMock).resizeThreadPool(6) @@ -263,7 +263,7 @@ class DynamicBrokerConfigTest { def testReconfigurableValidation(): Unit = { val origProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val config = KafkaConfig(origProps) - val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaConfig.BrokerIdProp, "some.prop") + val invalidReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, KafkaServerConfigs.BROKER_ID_CONFIG, "some.prop") val validReconfigurableProps = Set(CleanerConfig.LOG_CLEANER_THREADS_PROP, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "some.prop") def createReconfigurable(configs: Set[String]) = new Reconfigurable { @@ -616,15 +616,15 @@ class DynamicBrokerConfigTest { when(zkClient.getEntityConfigs(anyString(), anyString())).thenReturn(new java.util.Properties()) val initialProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 9092) - initialProps.remove(KafkaConfig.BackgroundThreadsProp) + initialProps.remove(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG) val oldConfig = KafkaConfig.fromProps(initialProps) val dynamicBrokerConfig = new DynamicBrokerConfig(oldConfig) dynamicBrokerConfig.initialize(Some(zkClient), None) dynamicBrokerConfig.addBrokerReconfigurable(new TestDynamicThreadPool) val newprops = new Properties() - newprops.put(KafkaConfig.NumIoThreadsProp, "10") - newprops.put(KafkaConfig.BackgroundThreadsProp, "100") + newprops.put(KafkaServerConfigs.NUM_IO_THREADS_CONFIG, "10") + newprops.put(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, "100") dynamicBrokerConfig.updateBrokerConfig(0, newprops) } @@ -639,7 +639,7 @@ class DynamicBrokerConfigTest { var newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "9999") - newProps.put(KafkaConfig.MessageMaxBytesProp, "2222") + newProps.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "2222") config.dynamicConfig.updateDefaultConfig(newProps) assertEquals(9999, config.maxConnections) @@ -647,7 +647,7 @@ class DynamicBrokerConfigTest { newProps = new Properties() newProps.put(SocketServerConfigs.MAX_CONNECTIONS_CONFIG, "INVALID_INT") - newProps.put(KafkaConfig.MessageMaxBytesProp, "1111") + newProps.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, "1111") config.dynamicConfig.updateDefaultConfig(newProps) // Invalid value should be skipped and reassigned as default value @@ -846,8 +846,8 @@ class TestDynamicThreadPool() extends BrokerReconfigurable { } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - assertEquals(Defaults.NUM_IO_THREADS, oldConfig.numIoThreads) - assertEquals(Defaults.BACKGROUND_THREADS, oldConfig.backgroundThreads) + assertEquals(KafkaServerConfigs.NUM_IO_THREADS_DEFAULT, oldConfig.numIoThreads) + assertEquals(KafkaServerConfigs.BACKGROUND_THREADS_DEFAULT, oldConfig.backgroundThreads) assertEquals(10, newConfig.numIoThreads) assertEquals(100, newConfig.backgroundThreads) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index 1c5d0a76f0f8a..b0be74a8f293c 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -23,6 +23,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} +import org.apache.kafka.server.config.KafkaServerConfigs import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -74,7 +75,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest { override protected def brokerPropertyOverrides(properties: Properties): Unit = { super.brokerPropertyOverrides(properties) - properties.put(KafkaConfig.FetchMaxBytes, "1024") + properties.put(KafkaServerConfigs.FETCH_MAX_BYTES_CONFIG, "1024") } private def createTopics(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index ce7acfe821f8d..bc5c88b052ce2 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -1180,7 +1180,7 @@ class KafkaApisTest extends Logging { val requestTimeout = 10 val topicConfigOverride = mutable.Map.empty[String, String] - topicConfigOverride.put(KafkaConfig.RequestTimeoutMsProp, requestTimeout.toString) + topicConfigOverride.put(KafkaServerConfigs.REQUEST_TIMEOUT_MS_CONFIG, requestTimeout.toString) val groupId = "group" val topicName = @@ -3155,7 +3155,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) when(groupCoordinator.completeTransaction( @@ -3164,7 +3164,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.ABORT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = @@ -3173,7 +3173,7 @@ class KafkaApisTest extends Logging { ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( - ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong), + ArgumentMatchers.eq(KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT.toLong), ArgumentMatchers.eq(-1), ArgumentMatchers.eq(true), ArgumentMatchers.eq(AppendOrigin.COORDINATOR), @@ -3274,7 +3274,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) + ArgumentMatchers.eq(Duration.ofMillis(KafkaServerConfigs.REQUEST_TIMEOUT_MS_DEFAULT)) )).thenReturn(FutureUtils.failedFuture[Void](error.exception())) kafkaApis = createKafkaApis(overrideProperties = Map( GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG -> "true" diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 8513854be7b4f..ba99a8d5616ba 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -40,13 +40,14 @@ import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1} -import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} +import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.storage.internals.log.CleanerConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable + import scala.annotation.nowarn import scala.jdk.CollectionConverters._ @@ -158,7 +159,7 @@ class KafkaConfigTest { val port = 9999 val hostName = "fake-host" val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"PLAINTEXT://$hostName:$port") val serverConfig = KafkaConfig.fromProps(props) @@ -189,7 +190,7 @@ class KafkaConfigTest { @Test def testDuplicateListeners(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // listeners with duplicate port @@ -215,7 +216,7 @@ class KafkaConfigTest { @Test def testIPv4AndIPv6SamePortListeners(): Unit = { val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, "1") + props.put(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://[::1]:9092,SSL://[::1]:9092") @@ -455,7 +456,7 @@ class KafkaConfigTest { @Test def testControllerListenerNameDoesNotMapToPlaintextByDefaultForNonKRaft(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9092") assertBadConfigContainingMessage(props, @@ -468,7 +469,7 @@ class KafkaConfigTest { @Test def testBadListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "BAD://localhost:9091") @@ -478,7 +479,7 @@ class KafkaConfigTest { @Test def testListenerNamesWithAdvertisedListenerUnset(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CLIENT://localhost:9091,REPLICATION://localhost:9092,INTERNAL://localhost:9093") @@ -502,7 +503,7 @@ class KafkaConfigTest { @Test def testListenerAndAdvertisedListenerNames(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "EXTERNAL://localhost:9091,INTERNAL://localhost:9093") @@ -533,7 +534,7 @@ class KafkaConfigTest { @Test def testListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091,REPLICATION://localhost:9092") @@ -544,7 +545,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameMissingFromListenerSecurityProtocolMap(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -555,7 +556,7 @@ class KafkaConfigTest { @Test def testInterBrokerListenerNameAndSecurityProtocolSet(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9091") @@ -567,7 +568,7 @@ class KafkaConfigTest { @Test def testCaseInsensitiveListenerProtocol(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092") val config = KafkaConfig.fromProps(props) @@ -582,7 +583,7 @@ class KafkaConfigTest { @Test def testListenerDefaults(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") // configuration with no listeners @@ -596,7 +597,7 @@ class KafkaConfigTest { @Test def testVersionConfiguration(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") val conf = KafkaConfig.fromProps(props) assertEquals(MetadataVersion.latestProduction, conf.interBrokerProtocolVersion) @@ -706,31 +707,31 @@ class KafkaConfigTest { @Test def testInvalidCompressionType(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "abc") + props.setProperty(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, "abc") assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidGzipCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "gzip") - props.setProperty(KafkaConfig.CompressionGzipLevelProp, (GzipCompression.MAX_LEVEL + 1).toString) + props.setProperty(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, "gzip") + props.setProperty(KafkaServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG, (GzipCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidLz4CompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "lz4") - props.setProperty(KafkaConfig.CompressionLz4LevelProp, (Lz4Compression.MAX_LEVEL + 1).toString) + props.setProperty(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, "lz4") + props.setProperty(KafkaServerConfigs.COMPRESSION_LZ_4_LEVEL_CONFIG, (Lz4Compression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @Test def testInvalidZstdCompressionLevel(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) - props.setProperty(KafkaConfig.CompressionTypeProp, "zstd") - props.setProperty(KafkaConfig.CompressionZstdLevelProp, (ZstdCompression.MAX_LEVEL + 1).toString) + props.setProperty(KafkaServerConfigs.COMPRESSION_TYPE_CONFIG, "zstd") + props.setProperty(KafkaServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG, (ZstdCompression.MAX_LEVEL + 1).toString) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) } @@ -820,16 +821,16 @@ class KafkaConfigTest { case ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") case ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean") - case KafkaConfig.BrokerIdProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.NumNetworkThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumIoThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.BackgroundThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.QueuedMaxRequestsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.NumReplicaAlterLogDirsThreadsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.QueuedMaxBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.BROKER_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.NUM_IO_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.BACKGROUND_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.QUEUED_MAX_REQUESTS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.QUEUED_MAX_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.REQUEST_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") // KRaft mode configs case KRaftConfigs.PROCESS_ROLES_CONFIG => // ignore @@ -845,7 +846,7 @@ class KafkaConfigTest { case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string case KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.AuthorizerClassNameProp => //ignore string + case KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG => //ignore string case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string case SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -909,9 +910,9 @@ class KafkaConfigTest { case ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") - case KafkaConfig.ControlledShutdownMaxRetriesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownRetryBackoffMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") - case KafkaConfig.ControlledShutdownEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case KafkaServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") + case KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") case GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number") @@ -934,18 +935,18 @@ class KafkaConfigTest { case TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0", "-2") case QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DeleteTopicEnableProp => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") + case KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_boolean", "0") case MetricConfigs.METRIC_NUM_SAMPLES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string + case KafkaServerConfigs.BROKER_RACK_CONFIG => // ignore string - case KafkaConfig.CompressionGzipLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionLz4LevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.CompressionZstdLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) + case KafkaServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.COMPRESSION_LZ_4_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case KafkaServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1) - case KafkaConfig.RackProp => // ignore string //SSL Configs case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG => case KafkaSecurityConfigs.CONNECTIONS_MAX_REAUTH_MS_CONFIG => @@ -1016,11 +1017,11 @@ class KafkaConfigTest { case PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0") //delegation token configs - case KafkaConfig.DelegationTokenSecretKeyAliasProp => // ignore - case KafkaConfig.DelegationTokenSecretKeyProp => // ignore - case KafkaConfig.DelegationTokenMaxLifeTimeProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryTimeMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") - case KafkaConfig.DelegationTokenExpiryCheckIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG => // ignore + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") + case DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "0") //Kafka Yammer metrics reporter configs case MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG => // ignore @@ -1180,9 +1181,9 @@ class KafkaConfigTest { defaults.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:2181") // For ZkConnectionTimeoutMs defaults.setProperty(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, "1234") - defaults.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") - defaults.setProperty(KafkaConfig.MaxReservedBrokerIdProp, "1") - defaults.setProperty(KafkaConfig.BrokerIdProp, "1") + defaults.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") + defaults.setProperty(KafkaServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "1") + defaults.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") defaults.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:1122") defaults.setProperty(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, "127.0.0.1:2, 127.0.0.2:3") defaults.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, "/tmp1,/tmp2") @@ -1215,7 +1216,7 @@ class KafkaConfigTest { assertEquals(24 * 60L * 60L * 1000L, config.delegationTokenExpiryTimeMs) assertEquals(1 * 60L * 1000L * 60, config.delegationTokenExpiryCheckIntervalMs) - defaults.setProperty(KafkaConfig.DelegationTokenSecretKeyProp, "1234567890") + defaults.setProperty(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "1234567890") val config1 = KafkaConfig.fromProps(defaults) assertEquals(true, config1.tokenAuthEnabled) } @@ -1436,7 +1437,7 @@ class KafkaConfigTest { // -1 is the default for both node.id and broker.id val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") assertFalse(isValidKafkaConfig(props)) } @@ -1473,7 +1474,7 @@ class KafkaConfigTest { props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, negativeTwoNodeId.toString) - props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, negativeTwoNodeId.toString) assertFalse(isValidKafkaConfig(props)) } @@ -1486,7 +1487,7 @@ class KafkaConfigTest { val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") KafkaConfig.fromProps(props) } @@ -1496,7 +1497,7 @@ class KafkaConfigTest { val props = TestUtils.createBrokerConfig(-1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") assertFalse(isValidKafkaConfig(props)) } @@ -1564,9 +1565,9 @@ class KafkaConfigTest { @Test def testPopulateSynonymsOnMapWithoutNodeId(): Unit = { val input = new util.HashMap[String, String]() - input.put(KafkaConfig.BrokerIdProp, "4") + input.put(KafkaServerConfigs.BROKER_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(KafkaServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1576,7 +1577,7 @@ class KafkaConfigTest { val input = new util.HashMap[String, String]() input.put(KRaftConfigs.NODE_ID_CONFIG, "4") val expectedOutput = new util.HashMap[String, String]() - expectedOutput.put(KafkaConfig.BrokerIdProp, "4") + expectedOutput.put(KafkaServerConfigs.BROKER_ID_CONFIG, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4") assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) } @@ -1584,7 +1585,7 @@ class KafkaConfigTest { @Test def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = { val props = new Properties() - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2") assertEquals("You must set `node.id` to the same value as `broker.id`.", assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) @@ -1605,13 +1606,13 @@ class KafkaConfigTest { val props = new Properties() props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL") - props.setProperty(KafkaConfig.BrokerIdProp, "3") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "3") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093") val config = KafkaConfig.fromProps(props) assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(KafkaServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1632,7 +1633,7 @@ class KafkaConfigTest { assertEquals(3, config.brokerId) assertEquals(3, config.nodeId) val originals = config.originals() - assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) + assertEquals("3", originals.get(KafkaServerConfigs.BROKER_ID_CONFIG)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG)) } @@ -1649,9 +1650,9 @@ class KafkaConfigTest { def testInvalidAuthorizerClassName(): Unit = { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181) val configs = new util.HashMap[Object, Object](props) - configs.put(KafkaConfig.AuthorizerClassNameProp, null) + configs.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, null) val ce = assertThrows(classOf[ConfigException], () => KafkaConfig.apply(configs)) - assertTrue(ce.getMessage.contains(KafkaConfig.AuthorizerClassNameProp)) + assertTrue(ce.getMessage.contains(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)) } @Test @@ -1678,7 +1679,7 @@ class KafkaConfigTest { def testEarlyStartListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL,INTERNAL2") + props.setProperty(KafkaServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL,INTERNAL2") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "INTERNAL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "INTERNAL:PLAINTEXT,INTERNAL2:PLAINTEXT,CONTROLLER:PLAINTEXT") @@ -1693,7 +1694,7 @@ class KafkaConfigTest { def testEarlyStartListenersMustBeListeners(): Unit = { val props = new Properties() props.putAll(kraftProps()) - props.setProperty(KafkaConfig.EarlyStartListenersProp, "INTERNAL") + props.setProperty(KafkaServerConfigs.EARLY_START_LISTENERS_CONFIG, "INTERNAL") assertEquals("early.start.listeners contains listener INTERNAL, but this is not " + "contained in listeners or controller.listener.names", assertThrows(classOf[ConfigException], () => new KafkaConfig(props)).getMessage) @@ -1765,7 +1766,7 @@ class KafkaConfigTest { KafkaConfig.fromProps(props) // Check that we allow authorizer to be set - props.setProperty(KafkaConfig.AuthorizerClassNameProp, classOf[AclAuthorizer].getCanonicalName) + props.setProperty(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[AclAuthorizer].getCanonicalName) KafkaConfig.fromProps(props) // Don't allow migration startup with an older IBP diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala index a27a49f9b2aa9..f63edb38d9197 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala @@ -21,6 +21,7 @@ import kafka.metrics.KafkaMetricsReporter import kafka.utils.{CoreUtils, TestUtils, VerifiableProperties} import kafka.server.QuorumTestHarness import org.apache.kafka.common.{ClusterResource, ClusterResourceListener} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.test.MockMetricsReporter import org.junit.jupiter.api.Assertions._ @@ -61,7 +62,7 @@ object KafkaMetricReporterClusterIdTest { // Because this code is run during the test setUp phase, if we throw an exception here, // it just results in the test itself being declared "not found" rather than failing. // So we track an error message which we will check later in the test body. - val brokerId = configs.get(KafkaConfig.BrokerIdProp) + val brokerId = configs.get(KafkaServerConfigs.BROKER_ID_CONFIG) if (brokerId == null) setupError.compareAndSet("", "No value was set for the broker id.") else if (!brokerId.isInstanceOf[String]) @@ -85,8 +86,8 @@ class KafkaMetricReporterClusterIdTest extends QuorumTestHarness { val props = TestUtils.createBrokerConfig(1, zkConnect) props.setProperty(MetricConfigs.KAFKA_METRICS_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockKafkaMetricsReporter") props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricReporterClusterIdTest$MockBrokerMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "-1") + props.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "-1") config = KafkaConfig.fromProps(props) server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 9886b1e056f4a..3830008533a29 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -20,6 +20,7 @@ import java.util import java.util.concurrent.atomic.AtomicReference import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsReporter} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Assertions._ @@ -72,8 +73,8 @@ class KafkaMetricsReporterTest extends QuorumTestHarness { super.setUp(testInfo) val props = TestUtils.createBrokerConfig(1, zkConnectOrNull) props.setProperty(MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG, "kafka.server.KafkaMetricsReporterTest$MockMetricsReporter") - props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") - props.setProperty(KafkaConfig.BrokerIdProp, "1") + props.setProperty(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "true") + props.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "1") config = KafkaConfig.fromProps(props) broker = createBroker(config, threadNamePrefix = Option(this.getClass.getName)) broker.startup() diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 7314d9ed612b7..2c498f5f479ed 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -45,7 +45,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.network.Session import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} -import org.apache.kafka.server.config.{KafkaSecurityConfigs, QuotaConfigs} +import org.apache.kafka.server.config.{KafkaSecurityConfigs, KafkaServerConfigs, QuotaConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -78,17 +78,17 @@ class RequestQuotaTest extends BaseRequestTest { private val tasks = new ListBuffer[Task] override def brokerPropertyOverrides(properties: Properties): Unit = { - properties.put(KafkaConfig.ControlledShutdownEnableProp, "false") + properties.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, "false") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") properties.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.put(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, "100") properties.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") properties.put(KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[RequestQuotaTest.TestPrincipalBuilder].getName) - properties.put(KafkaConfig.UnstableApiVersionsEnableProp, "true") + properties.put(KafkaServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, "true") if (isKRaftTest()) { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.KraftTestAuthorizer].getName) } else { - properties.put(KafkaConfig.AuthorizerClassNameProp, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) + properties.put(KafkaServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG, classOf[RequestQuotaTest.ZkTestAuthorizer].getName) } } diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index a4fa2b315e5be..fe07a5bbce679 100755 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -19,6 +19,7 @@ package kafka.server import kafka.utils.TestUtils import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, PropertiesUtils} +import org.apache.kafka.server.config.KafkaServerConfigs import org.apache.zookeeper.KeeperException.NodeExistsException import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -87,9 +88,9 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { @Test def testDisableGeneratedBrokerId(): Unit = { val props3 = TestUtils.createBrokerConfig(3, zkConnect) - props3.put(KafkaConfig.BrokerIdGenerationEnableProp, "false") + props3.put(KafkaServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG, "false") // Set reserve broker ids to cause collision and ensure disabling broker id generation ignores the setting - props3.put(KafkaConfig.MaxReservedBrokerIdProp, "0") + props3.put(KafkaServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG, "0") val config3 = KafkaConfig.fromProps(props3) val server3 = createServer(config3, threadNamePrefix = Option(this.getClass.getName)) servers = Seq(server3) @@ -151,14 +152,14 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness { servers = Seq(serverA) // adjust the broker config and start again - propsB.setProperty(KafkaConfig.BrokerIdProp, "2") + propsB.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "2") val serverB2 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) val startupException = assertThrows(classOf[RuntimeException], () => serverB2.startup()) assertTrue(startupException.getMessage.startsWith("Stored node id 1 doesn't match previous node id 2"), "Unexpected exception message " + startupException.getMessage) serverB2.config.logDirs.foreach(logDir => Utils.delete(new File(logDir))) - propsB.setProperty(KafkaConfig.BrokerIdProp, "3") + propsB.setProperty(KafkaServerConfigs.BROKER_ID_CONFIG, "3") val serverB3 = new KafkaServer(KafkaConfig.fromProps(propsB), threadNamePrefix = Option(this.getClass.getName)) serverB3.startup() diff --git a/core/src/test/scala/unit/kafka/server/ServerTest.scala b/core/src/test/scala/unit/kafka/server/ServerTest.scala index 8c36aa3464a00..4e060a9360280 100644 --- a/core/src/test/scala/unit/kafka/server/ServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerTest.scala @@ -17,11 +17,10 @@ package kafka.server import java.util.Properties - import org.apache.kafka.common.Uuid import org.apache.kafka.common.metrics.MetricsContext import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, KafkaServerConfigs, ZkConfigs} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -55,7 +54,7 @@ class ServerTest { val clusterId = Uuid.randomUuid().toString val props = new Properties() - props.put(KafkaConfig.BrokerIdProp, brokerId.toString) + props.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId.toString) props.put(ZkConfigs.ZK_CONNECT_CONFIG, "127.0.0.1:0") val config = KafkaConfig.fromProps(props) diff --git a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala index f41487c5d4ad3..f22ecd1e98374 100644 --- a/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala +++ b/core/src/test/scala/unit/kafka/server/TopicIdWithOldInterBrokerProtocolTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse, FetchRequest, FetchResponse, MetadataRequest, MetadataResponse} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.common.MetadataVersion.IBP_2_7_IV0 -import org.apache.kafka.server.config.ReplicationConfigs +import org.apache.kafka.server.config.{KafkaServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} @@ -41,7 +41,7 @@ class TopicIdWithOldInterBrokerProtocolTest extends BaseRequestTest { properties.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, IBP_2_7_IV0.toString) properties.setProperty(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "1") properties.setProperty(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, "2") - properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + properties.setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, s"rack/${properties.getProperty(KafkaServerConfigs.BROKER_ID_CONFIG)}") } @BeforeEach diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala index 896e7169671ff..794de40e0a380 100644 --- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala +++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala @@ -32,7 +32,7 @@ import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.raft.QuorumConfig -import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} +import org.apache.kafka.server.config.{KRaftConfigs, KafkaServerConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest @@ -464,7 +464,7 @@ Found problem: val propsStream = Files.newOutputStream(propsFile.toPath) try { properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, TestUtils.tempDir().toString) - properties.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, enableUnstable.toString) + properties.setProperty(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, enableUnstable.toString) properties.store(propsStream, "config.props") } finally { propsStream.close() diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 8ba2ea00cdd8e..2333888220fd8 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -304,18 +304,18 @@ object TestUtils extends Logging { }.mkString(",") val props = new Properties - props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") + props.put(KafkaServerConfigs.UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG, "true") if (zkConnect == null) { props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString) - props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + props.put(KafkaServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER") props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, protocolAndPorts. map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT") } else { - if (nodeId >= 0) props.put(KafkaConfig.BrokerIdProp, nodeId.toString) + if (nodeId >= 0) props.put(KafkaServerConfigs.BROKER_ID_CONFIG, nodeId.toString) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) } if (logDirCount > 1) { @@ -341,20 +341,20 @@ object TestUtils extends Logging { } props.put(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG, "1500") props.put(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG, "1500") - props.put(KafkaConfig.ControlledShutdownEnableProp, enableControlledShutdown.toString) - props.put(KafkaConfig.DeleteTopicEnableProp, enableDeleteTopic.toString) + props.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, enableControlledShutdown.toString) + props.put(KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, enableDeleteTopic.toString) props.put(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG, "1000") - props.put(KafkaConfig.ControlledShutdownRetryBackoffMsProp, "100") + props.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG, "100") props.put(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, "2097152") props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, "1") if (!props.containsKey(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG)) props.put(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, "5") if (!props.containsKey(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG)) props.put(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, "0") - rack.foreach(props.put(KafkaConfig.RackProp, _)) + rack.foreach(props.put(KafkaServerConfigs.BROKER_RACK_CONFIG, _)) // Reduce number of threads per broker - props.put(KafkaConfig.NumNetworkThreadsProp, "2") - props.put(KafkaConfig.BackgroundThreadsProp, "2") + props.put(KafkaServerConfigs.NUM_NETWORK_THREADS_CONFIG, "2") + props.put(KafkaServerConfigs.BACKGROUND_THREADS_CONFIG, "2") if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) }) props ++= sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId") @@ -367,13 +367,13 @@ object TestUtils extends Logging { } if (enableToken) - props.put(KafkaConfig.DelegationTokenSecretKeyProp, "secretkey") + props.put(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG, "secretkey") props.put(ServerLogConfigs.NUM_PARTITIONS_CONFIG, numPartitions.toString) props.put(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG, defaultReplicationFactor.toString) if (enableFetchFromFollower) { - props.put(KafkaConfig.RackProp, nodeId.toString) + props.put(KafkaServerConfigs.BROKER_RACK_CONFIG, nodeId.toString) props.put(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG, "org.apache.kafka.common.replica.RackAwareReplicaSelector") } props diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java index 1c5e499145fcf..8d9f63506fa25 100644 --- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java +++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/metadata/MetadataRequestBenchmark.java @@ -31,7 +31,6 @@ import kafka.server.FetchManager; import kafka.server.KafkaApis; import kafka.server.KafkaConfig; -import kafka.server.KafkaConfig$; import kafka.server.MetadataCache; import kafka.server.QuotaFactory; import kafka.server.ReplicaManager; @@ -62,6 +61,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator; import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.MetadataVersion; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.mockito.Mockito; import org.openjdk.jmh.annotations.Benchmark; @@ -178,7 +178,7 @@ private List endpoints(final int brokerId) { private KafkaApis createKafkaApis() { Properties kafkaProps = new Properties(); kafkaProps.put(ZkConfigs.ZK_CONNECT_CONFIG, "zk"); - kafkaProps.put(KafkaConfig$.MODULE$.BrokerIdProp(), brokerId + ""); + kafkaProps.put(KafkaServerConfigs.BROKER_ID_CONFIG, brokerId + ""); KafkaConfig config = new KafkaConfig(kafkaProps); return new KafkaApisBuilder(). setRequestChannel(requestChannel). diff --git a/server/src/main/java/org/apache/kafka/server/config/Defaults.java b/server/src/main/java/org/apache/kafka/server/config/Defaults.java deleted file mode 100644 index ce9aeab4a2456..0000000000000 --- a/server/src/main/java/org/apache/kafka/server/config/Defaults.java +++ /dev/null @@ -1,60 +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.server.config; - -import org.apache.kafka.clients.CommonClientConfigs; -public class Defaults { - /** ********* General Configuration *********/ - public static final boolean BROKER_ID_GENERATION_ENABLE = true; - public static final int MAX_RESERVED_BROKER_ID = 1000; - public static final int BROKER_ID = -1; - public static final int NUM_NETWORK_THREADS = 3; - public static final int NUM_IO_THREADS = 8; - public static final int BACKGROUND_THREADS = 10; - public static final int QUEUED_MAX_REQUESTS = 500; - public static final int QUEUED_MAX_REQUEST_BYTES = -1; - public static final boolean DELETE_TOPIC_ENABLE = true; - public static final int REQUEST_TIMEOUT_MS = 30000; - public static final long CONNECTION_SETUP_TIMEOUT_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS; - public static final long CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS; - - /** ********* KRaft mode configs *********/ - public static final int EMPTY_NODE_ID = -1; - public static final long SERVER_MAX_STARTUP_TIME_MS = Long.MAX_VALUE; - public static final int MIGRATION_METADATA_MIN_BATCH_SIZE = 200; - - /** ********* Authorizer Configuration *********/ - public static final String AUTHORIZER_CLASS_NAME = ""; - - /** ********* Controlled shutdown configuration *********/ - public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES = 3; - public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS = 5000; - public static final boolean CONTROLLED_SHUTDOWN_ENABLE = true; - - /** ********* Fetch Configuration *********/ - public static final int MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS = 1000; - public static final int FETCH_MAX_BYTES = 55 * 1024 * 1024; - - /** ********* Request Limit Configuration ***********/ - public static final int MAX_REQUEST_PARTITION_SIZE_LIMIT = 2000; - - - /** ********* Delegation Token Configuration *********/ - public static final long DELEGATION_TOKEN_MAX_LIFE_TIME_MS = 7 * 24 * 60 * 60 * 1000L; - public static final long DELEGATION_TOKEN_EXPIRY_TIME_MS = 24 * 60 * 60 * 1000L; - public static final long DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS = 1 * 60 * 60 * 1000L; -} diff --git a/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java b/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java new file mode 100644 index 0000000000000..9fbd98cf7b633 --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/config/DelegationTokenManagerConfigs.java @@ -0,0 +1,40 @@ +/* + * 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.server.config; + +public class DelegationTokenManagerConfigs { + /** ********* Delegation Token Configuration ****************/ + public static final String DELEGATION_TOKEN_SECRET_KEY_CONFIG = "delegation.token.secret.key"; + public static final String DELEGATION_TOKEN_SECRET_KEY_DOC = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " + + " If using Kafka with KRaft, the key must also be set across all controllers. " + + " If the key is not set or set to empty string, brokers will disable the delegation token support."; + + public static final String DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG = "delegation.token.master.key"; + public static final String DELEGATION_TOKEN_SECRET_KEY_ALIAS_DOC = "DEPRECATED: An alias for " + DELEGATION_TOKEN_SECRET_KEY_CONFIG + ", which should be used instead of this config."; + + public static final String DELEGATION_TOKEN_MAX_LIFETIME_CONFIG = "delegation.token.max.lifetime.ms"; + public static final long DELEGATION_TOKEN_MAX_LIFE_TIME_MS_DEFAULT = 7 * 24 * 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_MAX_LIFE_TIME_DOC = "The token has a maximum lifetime beyond which it cannot be renewed anymore. Default value 7 days."; + + public static final String DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG = "delegation.token.expiry.time.ms"; + public static final long DELEGATION_TOKEN_EXPIRY_TIME_MS_DEFAULT = 24 * 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_EXPIRY_TIME_MS_DOC = "The token validity time in milliseconds before the token needs to be renewed. Default value 1 day."; + + public static final String DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG = "delegation.token.expiry.check.interval.ms"; + public static final long DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_DEFAULT = 1 * 60 * 60 * 1000L; + public static final String DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_DOC = "Scan interval to remove expired delegation tokens."; +} diff --git a/server/src/main/java/org/apache/kafka/server/config/KafkaServerConfigs.java b/server/src/main/java/org/apache/kafka/server/config/KafkaServerConfigs.java new file mode 100644 index 0000000000000..e738d2897941d --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/config/KafkaServerConfigs.java @@ -0,0 +1,139 @@ +/* + * 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.server.config; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.server.authorizer.Authorizer; +public class KafkaServerConfigs { + /** ********* General Configuration ***********/ + public static final String RESERVED_BROKER_MAX_ID_CONFIG = "reserved.broker.max.id"; + public static final int RESERVED_BROKER_MAX_ID_DEFAULT = 1000; + public static final String RESERVED_BROKER_MAX_ID_DOC = "Max number that can be used for a broker.id"; + + public static final String BROKER_ID_GENERATION_ENABLE_CONFIG = "broker.id.generation.enable"; + public static final boolean BROKER_ID_GENERATION_ENABLE_DEFAULT = true; + public static final String BROKER_ID_GENERATION_ENABLE_DOC = "Enable automatic broker id generation on the server. When enabled the value configured for " + RESERVED_BROKER_MAX_ID_CONFIG + " should be reviewed."; + + + public static final String BROKER_ID_CONFIG = "broker.id"; + public static final int BROKER_ID_DEFAULT = -1; + public static final String BROKER_ID_DOC = "The broker id for this server. If unset, a unique broker id will be generated." + + "To avoid conflicts between ZooKeeper generated broker id's and user configured broker id's, generated broker ids " + + "start from " + RESERVED_BROKER_MAX_ID_CONFIG + " + 1."; + + public static final String MESSAGE_MAX_BYTES_CONFIG = "message.max.bytes"; + public static final String MESSAGE_MAX_BYTES_DOC = TopicConfig.MAX_MESSAGE_BYTES_DOC + + "This can be set per topic with the topic level " + TopicConfig.MAX_MESSAGE_BYTES_CONFIG + " config."; + + public static final String NUM_NETWORK_THREADS_CONFIG = "num.network.threads"; + 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 String NUM_IO_THREADS_CONFIG = "num.io.threads"; + public static final int NUM_IO_THREADS_DEFAULT = 8; + public static final String NUM_IO_THREADS_DOC = "The number of threads that the server uses for processing requests, which may include disk I/O"; + + public static final String BACKGROUND_THREADS_CONFIG = "background.threads"; + public static final int BACKGROUND_THREADS_DEFAULT = 10; + public static final String BACKGROUND_THREADS_DOC = "The number of threads to use for various background processing tasks"; + + public static final String NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG = "num.replica.alter.log.dirs.threads"; + public static final String NUM_REPLICA_ALTER_LOG_DIRS_THREADS_DOC = "The number of threads that can move replicas between log directories, which may include disk I/O"; + + public static final String QUEUED_MAX_REQUESTS_CONFIG = "queued.max.requests"; + public static final int QUEUED_MAX_REQUESTS_DEFAULT = 500; + public static final String QUEUED_MAX_REQUESTS_DOC = "The number of queued requests allowed for data-plane, before blocking the network threads"; + + public static final String QUEUED_MAX_BYTES_CONFIG = "queued.max.request.bytes"; + public static final int QUEUED_MAX_REQUEST_BYTES_DEFAULT = -1; + public static final String QUEUED_MAX_REQUEST_BYTES_DOC = "The number of queued bytes allowed before no more requests are read"; + + public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; + public static final int REQUEST_TIMEOUT_MS_DEFAULT = 30000; + public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG; + public static final long DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS; + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_DOC; + + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG; + public static final long SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS; + public static final String SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC = CommonClientConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_DOC; + + public static final String DELETE_TOPIC_ENABLE_CONFIG = "delete.topic.enable"; + public static final boolean DELETE_TOPIC_ENABLE_DEFAULT = true; + public static final String DELETE_TOPIC_ENABLE_DOC = "Enables delete topic. Delete topic through the admin tool will have no effect if this config is turned off"; + + public static final String COMPRESSION_TYPE_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG); + public static final String COMPRESSION_TYPE_DOC = "Specify the final compression type for a given topic. This configuration accepts the standard compression codecs " + + "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + + "'producer' which means retain the original compression codec set by the producer."; + + public static final String COMPRESSION_GZIP_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG); + public static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'gzip'."; + public static final String COMPRESSION_LZ_4_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG); + public static final String COMPRESSION_LZ_4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'lz4'."; + public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG); + public static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to 'zstd'."; + + /***************** rack configuration *************/ + public static final String BROKER_RACK_CONFIG = "broker.rack"; + public static final String BROKER_RACK_DOC = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: RACK1, us-east-1d"; + + /** ********* Controlled shutdown configuration ***********/ + public static final String CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG = "controlled.shutdown.max.retries"; + public static final int CONTROLLED_SHUTDOWN_MAX_RETRIES_DEFAULT = 3; + public static final String CONTROLLED_SHUTDOWN_MAX_RETRIES_DOC = "Controlled shutdown can fail for multiple reasons. This determines the number of retries when such failure happens"; + + public static final String CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG = "controlled.shutdown.retry.backoff.ms"; + public static final int CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DEFAULT = 5000; + public static final String CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_DOC = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying."; + + public static final String CONTROLLED_SHUTDOWN_ENABLE_CONFIG = "controlled.shutdown.enable"; + public static final boolean CONTROLLED_SHUTDOWN_ENABLE_DEFAULT = true; + public static final String CONTROLLED_SHUTDOWN_ENABLE_DOC = "Enable controlled shutdown of the server."; + + /** ********* Fetch Configuration **************/ + public static final String MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG = "max.incremental.fetch.session.cache.slots"; + public static final int MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DEFAULT = 1000; + public static final String MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_DOC = "The maximum number of total incremental fetch sessions that we will maintain. FetchSessionCache is sharded into 8 shards and the limit is equally divided among all shards. Sessions are allocated to each shard in round-robin. Only entries within a shard are considered eligible for eviction."; + + public static final String FETCH_MAX_BYTES_CONFIG = "fetch.max.bytes"; + public static final int FETCH_MAX_BYTES_DEFAULT = 55 * 1024 * 1024; + public static final String FETCH_MAX_BYTES_DOC = "The maximum number of bytes we will return for a fetch request. Must be at least 1024."; + + /** ********* Request Limit Configuration **************/ + public static final String MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG = "max.request.partition.size.limit"; + public static final int MAX_REQUEST_PARTITION_SIZE_LIMIT_DEFAULT = 2000; + public static final String MAX_REQUEST_PARTITION_SIZE_LIMIT_DOC = "The maximum number of partitions can be served in one request."; + + /** Internal Configurations **/ + public static final String UNSTABLE_API_VERSIONS_ENABLE_CONFIG = "unstable.api.versions.enable"; + public static final String UNSTABLE_METADATA_VERSIONS_ENABLE_CONFIG = "unstable.metadata.versions.enable"; + + /************* Authorizer Configuration ***********/ + public static final String AUTHORIZER_CLASS_NAME_CONFIG = "authorizer.class.name"; + public static final String AUTHORIZER_CLASS_NAME_DEFAULT = ""; + public static final String AUTHORIZER_CLASS_NAME_DOC = "The fully qualified name of a class that implements " + + Authorizer.class.getName() + " interface, which is used by the broker for authorization."; + public static final String EARLY_START_LISTENERS_CONFIG = "early.start.listeners"; + public static final String EARLY_START_LISTENERS_DOC = "A comma-separated list of listener names which may be started before the authorizer has finished " + + "initialization. This is useful when the authorizer is dependent on the cluster itself for bootstrapping, as is the case for " + + "the StandardAuthorizer (which stores ACLs in the metadata log.) By default, all listeners included in controller.listener.names " + + "will also be early start listeners. A listener should not appear in this list if it accepts external traffic."; +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java index 7d512c14f57f9..dc60c4d89cb06 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/HighAvailabilityTaskAssignorIntegrationTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.integration; import java.util.stream.Stream; -import kafka.server.KafkaConfig; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -31,6 +30,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; @@ -85,13 +85,13 @@ public class HighAvailabilityTaskAssignorIntegrationTest { new Properties(), asList( new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_0); + setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_0); }}, new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_1); + setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_1); }}, new Properties() {{ - setProperty(KafkaConfig.RackProp(), AssignmentTestUtils.RACK_2); + setProperty(KafkaServerConfigs.BROKER_RACK_CONFIG, AssignmentTestUtils.RACK_2); }} ) ); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java index 6ea41b1e76bb2..49411ba93caf2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedKafkaCluster.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.integration.utils; -import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.zk.EmbeddedZookeeper; import org.apache.kafka.clients.admin.Admin; @@ -26,6 +25,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.network.SocketServerConfigs; import org.apache.kafka.server.config.ConfigType; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.config.ServerLogConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.server.util.MockTime; @@ -116,7 +116,7 @@ public void start() throws IOException { brokerConfig.put(ZkConfigs.ZK_CONNECT_CONFIG, zKConnectString()); putIfAbsent(brokerConfig, SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:" + DEFAULT_BROKER_PORT); - putIfAbsent(brokerConfig, KafkaConfig.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, KafkaServerConfigs.DELETE_TOPIC_ENABLE_CONFIG, true); putIfAbsent(brokerConfig, CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP, 2 * 1024 * 1024L); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG, 0); putIfAbsent(brokerConfig, GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG, 0); @@ -126,7 +126,7 @@ public void start() throws IOException { putIfAbsent(brokerConfig, ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG, true); for (int i = 0; i < brokers.length; i++) { - brokerConfig.put(KafkaConfig.BrokerIdProp(), i); + brokerConfig.put(KafkaServerConfigs.BROKER_ID_CONFIG, i); log.debug("Starting a Kafka instance on {} ...", brokerConfig.get(SocketServerConfigs.LISTENERS_CONFIG)); final Properties effectiveConfig = new Properties(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index b1aca31563218..0f2dadfeeb041 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -29,6 +29,7 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.network.SocketServerConfigs; +import org.apache.kafka.server.config.KafkaServerConfigs; import org.apache.kafka.server.config.ZkConfigs; import org.apache.kafka.server.util.MockTime; import org.slf4j.Logger; @@ -92,11 +93,11 @@ public KafkaEmbedded(final Properties config, final MockTime time) throws IOExce */ private Properties effectiveConfigFrom(final Properties initialConfig) { final Properties effectiveConfig = new Properties(); - effectiveConfig.put(KafkaConfig.BrokerIdProp(), 0); + effectiveConfig.put(KafkaServerConfigs.BROKER_ID_CONFIG, 0); effectiveConfig.put(NUM_PARTITIONS_CONFIG, 1); effectiveConfig.put(AUTO_CREATE_TOPICS_ENABLE_CONFIG, true); - effectiveConfig.put(KafkaConfig.MessageMaxBytesProp(), 1000000); - effectiveConfig.put(KafkaConfig.ControlledShutdownEnableProp(), true); + effectiveConfig.put(KafkaServerConfigs.MESSAGE_MAX_BYTES_CONFIG, 1000000); + effectiveConfig.put(KafkaServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG, true); effectiveConfig.put(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG, 10000); effectiveConfig.putAll(initialConfig);