From 62ce551826192ef6137bc3ce670277f79bd3dee2 Mon Sep 17 00:00:00 2001 From: Omnia Ibrahim Date: Mon, 22 Jan 2024 09:29:11 -0500 Subject: [PATCH] KAFKA-15853: Move KafkaConfig.Defaults to server module (#15158) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Reviewers: Mickael Maison , Ismael Juma 
, David Jacot , Nikolay --- build.gradle | 25 + checkstyle/import-control-server.xml | 5 + .../scala/kafka/admin/ConfigCommand.scala | 10 +- .../transaction/TransactionLog.scala | 7 - .../transaction/TransactionStateManager.scala | 42 +- .../kafka/metrics/KafkaMetricsConfig.scala | 8 +- .../main/scala/kafka/server/KafkaConfig.scala | 609 ++++++------------ .../api/PlaintextAdminIntegrationTest.scala | 5 +- .../DynamicNumNetworkThreadsTest.scala | 9 +- .../scala/other/kafka/StressTestLog.scala | 5 +- .../other/kafka/TestLinearWriteSpeed.scala | 5 +- ...sConsumerGroupCommandIntegrationTest.scala | 5 +- .../kafka/cluster/AbstractPartitionTest.scala | 5 +- .../kafka/cluster/PartitionLockTest.scala | 5 +- .../unit/kafka/cluster/PartitionTest.scala | 37 +- .../TransactionCoordinatorTest.scala | 11 +- .../AbstractLogCleanerIntegrationTest.scala | 5 +- .../kafka/log/BrokerCompressionTest.scala | 5 +- .../kafka/log/LogCleanerManagerTest.scala | 9 +- .../scala/unit/kafka/log/LogCleanerTest.scala | 7 +- .../unit/kafka/log/LogConcurrencyTest.scala | 5 +- .../scala/unit/kafka/log/LogLoaderTest.scala | 11 +- .../scala/unit/kafka/log/LogManagerTest.scala | 5 +- .../scala/unit/kafka/log/LogSegmentTest.scala | 3 +- .../scala/unit/kafka/log/LogTestUtils.scala | 5 +- .../kafka/log/ProducerStateManagerTest.scala | 3 +- .../scala/unit/kafka/log/UnifiedLogTest.scala | 5 +- .../DelegationTokenManagerTest.scala | 7 +- .../server/DynamicBrokerConfigTest.scala | 9 +- .../unit/kafka/server/KafkaApisTest.scala | 10 +- .../kafka/tools/DumpLogSegmentsTest.scala | 5 +- .../kafka/utils/PasswordEncoderTest.scala | 17 +- .../unit/kafka/utils/SchedulerTest.scala | 5 +- .../scala/unit/kafka/utils/TestUtils.scala | 5 +- .../kafka/coordinator/group/OffsetConfig.java | 2 +- .../apache/kafka/server/config/Defaults.java | 278 ++++++++ settings.gradle | 1 + .../transaction/TransactionLogConfig.java | 26 + .../TransactionStateManagerConfig.java | 30 + 39 files changed, 692 insertions(+), 559 deletions(-) create mode 100644 server/src/main/java/org/apache/kafka/server/config/Defaults.java create mode 100644 transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionLogConfig.java create mode 100644 transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionStateManagerConfig.java diff --git a/build.gradle b/build.gradle index e105870ed64..2ca19c9a323 100644 --- a/build.gradle +++ b/build.gradle @@ -849,6 +849,9 @@ project(':server') { dependencies { implementation project(':clients') implementation project(':server-common') + implementation project(':group-coordinator') + implementation project(':transaction-coordinator') + implementation project(':raft') implementation libs.metrics implementation libs.slf4jApi @@ -926,6 +929,7 @@ project(':core') { implementation project(':server-common') implementation project(':group-coordinator') + implementation project(':transaction-coordinator') implementation project(':metadata') implementation project(':storage:storage-api') implementation project(':tools:tools-api') @@ -1363,6 +1367,27 @@ project(':group-coordinator') { srcJar.dependsOn 'processMessages' } +project(':transaction-coordinator') { + archivesBaseName = "kafka-transaction-coordinator" + + sourceSets { + main { + java { + srcDirs = ["src/generated/java", "src/main/java"] + } + } + test { + java { + srcDirs = ["src/generated/java", "src/test/java"] + } + } + } + + javadoc { + enabled = false + } +} + project(':examples') { archivesBaseName = "kafka-examples" diff --git a/checkstyle/import-control-server.xml b/checkstyle/import-control-server.xml index 7bf55852242..e6fe20db3a5 100644 --- a/checkstyle/import-control-server.xml +++ b/checkstyle/import-control-server.xml @@ -70,6 +70,11 @@ + + + + + diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 079d35e7b23..a8ca5960528 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import java.util.{Collections, Properties} import joptsimple._ import kafka.server.DynamicConfig.QuotaConfigs -import kafka.server.{Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig} +import kafka.server.{DynamicBrokerConfig, DynamicConfig, KafkaConfig} import kafka.utils.{Exit, Logging, PasswordEncoder} import kafka.utils.Implicits._ import kafka.zk.{AdminZkClient, KafkaZkClient} @@ -35,7 +35,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} -import org.apache.kafka.server.config.{ConfigEntityName, ConfigType} +import org.apache.kafka.server.config.{ConfigEntityName, ConfigType, Defaults} import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} import org.apache.kafka.storage.internals.log.LogConfig import org.apache.zookeeper.client.ZKClientConfig @@ -216,9 +216,9 @@ object ConfigCommand extends Logging { throw new IllegalArgumentException("Password encoder secret not specified")) PasswordEncoder.encrypting(new Password(encoderSecret), None, - encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderCipherAlgorithmProp, Defaults.PasswordEncoderCipherAlgorithm), - encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderKeyLength), - encoderConfigs.get(KafkaConfig.PasswordEncoderIterationsProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderIterations)) + encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderCipherAlgorithmProp, Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM), + encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PASSWORD_ENCODER_KEY_LENGTH), + encoderConfigs.get(KafkaConfig.PasswordEncoderIterationsProp).map(_.toInt).getOrElse(Defaults.PASSWORD_ENCODER_ITERATIONS)) } /** diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala index f07e7293fc2..8716630b4ba 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionLog.scala @@ -38,13 +38,6 @@ import scala.jdk.CollectionConverters._ */ object TransactionLog { - // log-level config default values and enforced values - val DefaultNumPartitions: Int = 50 - val DefaultSegmentBytes: Int = 100 * 1024 * 1024 - val DefaultReplicationFactor: Short = 3.toShort - val DefaultMinInSyncReplicas: Int = 2 - val DefaultLoadBufferSize: Int = 5 * 1024 * 1024 - // enforce always using // 1. cleanup policy = compact // 2. compression = none diff --git a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala index af46bfcc95c..44c7de5df86 100644 --- a/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala +++ b/core/src/main/scala/kafka/coordinator/transaction/TransactionStateManager.scala @@ -18,10 +18,9 @@ package kafka.coordinator.transaction import java.nio.ByteBuffer import java.util.Properties -import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.server.{Defaults, ReplicaManager, RequestLocal} +import kafka.server.{ReplicaManager, RequestLocal} import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils.{Logging, Pool} import kafka.utils.Implicits._ @@ -36,6 +35,8 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse 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.{TransactionLogConfig, TransactionStateManagerConfig} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation} @@ -44,17 +45,6 @@ import scala.jdk.CollectionConverters._ import scala.collection.mutable -object TransactionStateManager { - // default transaction management config values - val DefaultTransactionsMaxTimeoutMs: Int = TimeUnit.MINUTES.toMillis(15).toInt - val DefaultTransactionalIdExpirationMs: Int = TimeUnit.DAYS.toMillis(7).toInt - val DefaultAbortTimedOutTransactionsIntervalMs: Int = TimeUnit.SECONDS.toMillis(10).toInt - val DefaultRemoveExpiredTransactionalIdsIntervalMs: Int = TimeUnit.HOURS.toMillis(1).toInt - - val MetricsGroup: String = "transaction-coordinator-metrics" - val LoadTimeSensor: String = "TransactionsPartitionLoadTime" -} - /** * Transaction state manager is part of the transaction coordinator, it manages: * @@ -100,13 +90,13 @@ class TransactionStateManager(brokerId: Int, @volatile private var transactionTopicPartitionCount: Int = _ /** setup metrics*/ - private val partitionLoadSensor = metrics.sensor(TransactionStateManager.LoadTimeSensor) + private val partitionLoadSensor = metrics.sensor(TransactionStateManagerConfig.LOAD_TIME_SENSOR) partitionLoadSensor.add(metrics.metricName("partition-load-time-max", - TransactionStateManager.MetricsGroup, + TransactionStateManagerConfig.METRICS_GROUP, "The max time it took to load the partitions in the last 30sec"), new Max()) partitionLoadSensor.add(metrics.metricName("partition-load-time-avg", - TransactionStateManager.MetricsGroup, + TransactionStateManagerConfig.METRICS_GROUP, "The avg time it took to load the partitions in the last 30sec"), new Avg()) // visible for testing only @@ -808,16 +798,16 @@ private[transaction] case class TxnMetadataCacheEntry(coordinatorEpoch: Int, private[transaction] case class CoordinatorEpochAndTxnMetadata(coordinatorEpoch: Int, transactionMetadata: TransactionMetadata) -private[transaction] case class TransactionConfig(transactionalIdExpirationMs: Int = TransactionStateManager.DefaultTransactionalIdExpirationMs, - transactionMaxTimeoutMs: Int = TransactionStateManager.DefaultTransactionsMaxTimeoutMs, - transactionLogNumPartitions: Int = TransactionLog.DefaultNumPartitions, - transactionLogReplicationFactor: Short = TransactionLog.DefaultReplicationFactor, - transactionLogSegmentBytes: Int = TransactionLog.DefaultSegmentBytes, - transactionLogLoadBufferSize: Int = TransactionLog.DefaultLoadBufferSize, - transactionLogMinInsyncReplicas: Int = TransactionLog.DefaultMinInSyncReplicas, - abortTimedOutTransactionsIntervalMs: Int = TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs, - removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManager.DefaultRemoveExpiredTransactionalIdsIntervalMs, - requestTimeoutMs: Int = Defaults.RequestTimeoutMs) +private[transaction] case class TransactionConfig(transactionalIdExpirationMs: Int = TransactionStateManagerConfig.DEFAULT_TRANSACTIONAL_ID_EXPIRATION_MS, + transactionMaxTimeoutMs: Int = TransactionStateManagerConfig.DEFAULT_TRANSACTIONS_MAX_TIMEOUT_MS, + transactionLogNumPartitions: Int = TransactionLogConfig.DEFAULT_NUM_PARTITIONS, + transactionLogReplicationFactor: Short = TransactionLogConfig.DEFAULT_REPLICATION_FACTOR, + transactionLogSegmentBytes: Int = TransactionLogConfig.DEFAULT_SEGMENT_BYTES, + transactionLogLoadBufferSize: Int = TransactionLogConfig.DEFAULT_LOAD_BUFFER_SIZE, + transactionLogMinInsyncReplicas: Int = TransactionLogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS, + abortTimedOutTransactionsIntervalMs: Int = TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS, + removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManagerConfig.DEFAULT_REMOVE_EXPIRED_TRANSACTIONAL_IDS_INTERVAL_MS, + requestTimeoutMs: Int = Defaults.REQUEST_TIMEOUT_MS) case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) { override def toString: String = { diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala index b13a1b9350f..64bc98454eb 100755 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsConfig.scala @@ -20,8 +20,10 @@ package kafka.metrics -import kafka.server.{Defaults, KafkaConfig} +import kafka.server.KafkaConfig import kafka.utils.{CoreUtils, VerifiableProperties} +import org.apache.kafka.server.config.Defaults + import scala.collection.Seq class KafkaMetricsConfig(props: VerifiableProperties) { @@ -31,11 +33,11 @@ class KafkaMetricsConfig(props: VerifiableProperties) { * classpath and will be instantiated at run-time. */ val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(KafkaConfig.KafkaMetricsReporterClassesProp, - Defaults.KafkaMetricReporterClasses)) + Defaults.KAFKA_METRIC_REPORTER_CLASSES)) /** * The metrics polling interval (in seconds). */ val pollingIntervalSecs: Int = props.getInt(KafkaConfig.KafkaMetricsPollingIntervalSecondsProp, - Defaults.KafkaMetricsPollingIntervalSeconds) + Defaults.KAFKA_METRICS_POLLING_INTERVAL_SECONDS) } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 6bbf4968bc5..fec8519c928 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,9 +19,8 @@ package kafka.server import java.{lang, util} import java.util.concurrent.TimeUnit -import java.util.{Collections, Locale, Properties} +import java.util.{Collections, Properties} import kafka.cluster.EndPoint -import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager} import kafka.security.authorizer.AuthorizerUtils import kafka.server.KafkaConfig.{ControllerListenerNamesProp, ListenerSecurityProtocolMapProp} import kafka.utils.CoreUtils.parseCsvList @@ -29,26 +28,23 @@ 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.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslClientAuth, SslConfigs, TopicConfig} +import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SecurityConfig, SslConfigs, TopicConfig} import org.apache.kafka.common.config.ConfigDef.{ConfigKey, ValidList} import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.types.Password -import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.{CompressionType, LegacyRecord, Records, TimestampType} import org.apache.kafka.common.security.auth.KafkaPrincipalSerde import org.apache.kafka.common.security.auth.SecurityProtocol -import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.Group.GroupType -import org.apache.kafka.coordinator.group.OffsetConfig -import org.apache.kafka.coordinator.group.assignor.{PartitionAssignor, RangeAssignor, UniformAssignor} +import org.apache.kafka.coordinator.group.assignor.PartitionAssignor import org.apache.kafka.raft.RaftConfig 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.{ClientQuotaManagerConfig, ReplicationQuotaManagerConfig, ServerTopicConfigSynonyms} +import org.apache.kafka.server.config.{Defaults, ServerTopicConfigSynonyms} import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig @@ -60,241 +56,6 @@ import scala.compat.java8.OptionConverters._ import scala.jdk.CollectionConverters._ import scala.collection.{Map, Seq} -object Defaults { - /** ********* Zookeeper Configuration ***********/ - val ZkSessionTimeoutMs = 18000 - val ZkEnableSecureAcls = false - val ZkMaxInFlightRequests = 10 - val ZkSslClientEnable = false - val ZkSslProtocol = "TLSv1.2" - val ZkSslEndpointIdentificationAlgorithm = "HTTPS" - val ZkSslCrlEnable = false - val ZkSslOcspEnable = false - - /** ********* General Configuration ***********/ - val BrokerIdGenerationEnable = true - val MaxReservedBrokerId = 1000 - val BrokerId = -1 - val NumNetworkThreads = 3 - val NumIoThreads = 8 - val BackgroundThreads = 10 - val QueuedMaxRequests = 500 - val QueuedMaxRequestBytes = -1 - val InitialBrokerRegistrationTimeoutMs = 60000 - val BrokerHeartbeatIntervalMs = 2000 - val BrokerSessionTimeoutMs = 9000 - val MetadataSnapshotMaxNewRecordBytes = 20 * 1024 * 1024 - val MetadataSnapshotMaxIntervalMs = TimeUnit.HOURS.toMillis(1) - val MetadataMaxIdleIntervalMs = 500 - val MetadataMaxRetentionBytes = 100 * 1024 * 1024 - val DeleteTopicEnable = true - - /** KRaft mode configs */ - val EmptyNodeId: Int = -1 - val ServerMaxStartupTimeMs = Long.MaxValue - - /************* Authorizer Configuration ***********/ - val AuthorizerClassName = "" - - /** ********* Socket Server Configuration ***********/ - val Listeners = "PLAINTEXT://:9092" - val ListenerSecurityProtocolMap: String = EndPoint.DefaultSecurityProtocolMap.map { case (listenerName, securityProtocol) => - s"${listenerName.value}:${securityProtocol.name}" - }.mkString(",") - - val SocketSendBufferBytes: Int = 100 * 1024 - val SocketReceiveBufferBytes: Int = 100 * 1024 - val SocketRequestMaxBytes: Int = 100 * 1024 * 1024 - val SocketListenBacklogSize: Int = 50 - val MaxConnectionsPerIp: Int = Int.MaxValue - val MaxConnectionsPerIpOverrides: String = "" - val MaxConnections: Int = Int.MaxValue - val MaxConnectionCreationRate: Int = Int.MaxValue - val ConnectionsMaxIdleMs = 10 * 60 * 1000L - val RequestTimeoutMs = 30000 - val ConnectionSetupTimeoutMs = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS - val ConnectionSetupTimeoutMaxMs = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS - val FailedAuthenticationDelayMs = 100 - - /** ********* Log Configuration ***********/ - val NumPartitions = 1 - val LogDir = "/tmp/kafka-logs" - val LogCleanupIntervalMs = 5 * 60 * 1000L - val LogCleanerThreads = 1 - val LogCleanerIoMaxBytesPerSecond = Double.MaxValue - val LogCleanerDedupeBufferSize = 128 * 1024 * 1024L - val LogCleanerIoBufferSize = 512 * 1024 - val LogCleanerDedupeBufferLoadFactor = 0.9d - val LogCleanerBackoffMs = 15 * 1000 - val LogCleanerEnable = true - val LogFlushOffsetCheckpointIntervalMs = 60000 - val LogFlushStartOffsetCheckpointIntervalMs = 60000 - val NumRecoveryThreadsPerDataDir = 1 - val AutoCreateTopicsEnable = true - - /** ********* Replication configuration ***********/ - val ControllerSocketTimeoutMs = RequestTimeoutMs - val DefaultReplicationFactor = 1 - val ReplicaLagTimeMaxMs = 30000L - val ReplicaSocketTimeoutMs = 30 * 1000 - val ReplicaSocketReceiveBufferBytes = 64 * 1024 - val ReplicaFetchMaxBytes = 1024 * 1024 - val ReplicaFetchWaitMaxMs = 500 - val ReplicaFetchMinBytes = 1 - val ReplicaFetchResponseMaxBytes = 10 * 1024 * 1024 - val NumReplicaFetchers = 1 - val ReplicaFetchBackoffMs = 1000 - val ReplicaHighWatermarkCheckpointIntervalMs = 5000L - val FetchPurgatoryPurgeIntervalRequests = 1000 - val ProducerPurgatoryPurgeIntervalRequests = 1000 - val DeleteRecordsPurgatoryPurgeIntervalRequests = 1 - val AutoLeaderRebalanceEnable = true - val LeaderImbalancePerBrokerPercentage = 10 - val LeaderImbalanceCheckIntervalSeconds = 300 - val InterBrokerSecurityProtocol = SecurityProtocol.PLAINTEXT.toString - val InterBrokerProtocolVersion = MetadataVersion.latestProduction.version - - /** ********* Controlled shutdown configuration ***********/ - val ControlledShutdownMaxRetries = 3 - val ControlledShutdownRetryBackoffMs = 5000 - val ControlledShutdownEnable = true - - /** ********* Group coordinator configuration ***********/ - val GroupMinSessionTimeoutMs = 6000 - val GroupMaxSessionTimeoutMs = 1800000 - val GroupInitialRebalanceDelayMs = 3000 - val GroupMaxSize: Int = Int.MaxValue - - /** New group coordinator configs */ - val NewGroupCoordinatorEnable = false - val GroupCoordinatorRebalanceProtocols = List(GroupType.CLASSIC.toString).asJava - val GroupCoordinatorNumThreads = 1 - - /** Consumer group configs */ - val ConsumerGroupSessionTimeoutMs = 45000 - val ConsumerGroupMinSessionTimeoutMs = 45000 - val ConsumerGroupMaxSessionTimeoutMs = 60000 - val ConsumerGroupHeartbeatIntervalMs = 5000 - val ConsumerGroupMinHeartbeatIntervalMs = 5000 - val ConsumerGroupMaxHeartbeatIntervalMs = 15000 - val ConsumerGroupMaxSize = Int.MaxValue - val ConsumerGroupAssignors = List(classOf[UniformAssignor].getName, classOf[RangeAssignor].getName).asJava - - /** ********* Offset management configuration ***********/ - val OffsetMetadataMaxSize = OffsetConfig.DEFAULT_MAX_METADATA_SIZE - val OffsetsLoadBufferSize = OffsetConfig.DEFAULT_LOAD_BUFFER_SIZE - val OffsetsTopicReplicationFactor = OffsetConfig.DEFAULT_OFFSETS_TOPIC_REPLICATION_FACTOR - val OffsetsTopicPartitions: Int = OffsetConfig.DEFAULT_OFFSETS_TOPIC_NUM_PARTITIONS - val OffsetsTopicSegmentBytes: Int = OffsetConfig.DEFAULT_OFFSETS_TOPIC_SEGMENT_BYTES - val OffsetsTopicCompressionCodec: Int = OffsetConfig.DEFAULT_OFFSETS_TOPIC_COMPRESSION_TYPE.id - val OffsetsRetentionMinutes: Int = 7 * 24 * 60 - val OffsetsRetentionCheckIntervalMs: Long = OffsetConfig.DEFAULT_OFFSETS_RETENTION_CHECK_INTERVAL_MS - val OffsetCommitTimeoutMs = OffsetConfig.DEFAULT_OFFSET_COMMIT_TIMEOUT_MS - val OffsetCommitRequiredAcks = OffsetConfig.DEFAULT_OFFSET_COMMIT_REQUIRED_ACKS - - /** ********* Transaction management configuration ***********/ - val TransactionalIdExpirationMs = TransactionStateManager.DefaultTransactionalIdExpirationMs - val TransactionsMaxTimeoutMs = TransactionStateManager.DefaultTransactionsMaxTimeoutMs - val TransactionsTopicMinISR = TransactionLog.DefaultMinInSyncReplicas - val TransactionsLoadBufferSize = TransactionLog.DefaultLoadBufferSize - val TransactionsTopicReplicationFactor = TransactionLog.DefaultReplicationFactor - val TransactionsTopicPartitions = TransactionLog.DefaultNumPartitions - val TransactionsTopicSegmentBytes = TransactionLog.DefaultSegmentBytes - val TransactionsAbortTimedOutTransactionsCleanupIntervalMS = TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs - val TransactionsRemoveExpiredTransactionsCleanupIntervalMS = TransactionStateManager.DefaultRemoveExpiredTransactionalIdsIntervalMs - - val TransactionPartitionVerificationEnable = true - - val ProducerIdExpirationMs = 86400000 - val ProducerIdExpirationCheckIntervalMs = 600000 - - /** ********* Fetch Configuration **************/ - val MaxIncrementalFetchSessionCacheSlots = 1000 - val FetchMaxBytes = 55 * 1024 * 1024 - - /** ********* Quota Configuration ***********/ - val NumQuotaSamples: Int = ClientQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES - val QuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS - val NumReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES - val ReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS - val NumAlterLogDirsReplicationQuotaSamples: Int = ReplicationQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES - val AlterLogDirsReplicationQuotaWindowSizeSeconds: Int = ReplicationQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS - val NumControllerQuotaSamples: Int = ClientQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES - val ControllerQuotaWindowSizeSeconds: Int = ClientQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS - - /** ********* Kafka Metrics Configuration ***********/ - val MetricNumSamples = 2 - val MetricSampleWindowMs = 30000 - val MetricReporterClasses = "" - val MetricRecordingLevel = Sensor.RecordingLevel.INFO.toString - val AutoIncludeJmxReporter = true - - - /** ********* Kafka Yammer Metrics Reporter Configuration ***********/ - val KafkaMetricReporterClasses = "" - val KafkaMetricsPollingIntervalSeconds = 10 - - /** ********* Kafka Client Telemetry Metrics Configuration ***********/ - val ClientTelemetryMaxBytes = 1024 * 1024 - - /** ********* SSL configuration ***********/ - val SslProtocol = SslConfigs.DEFAULT_SSL_PROTOCOL - val SslEnabledProtocols = SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS - val SslKeystoreType = SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE - val SslTruststoreType = SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE - val SslKeyManagerAlgorithm = SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM - val SslTrustManagerAlgorithm = SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM - val SslEndpointIdentificationAlgorithm = SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM - val SslClientAuthentication = SslClientAuth.NONE.name().toLowerCase(Locale.ROOT) - val SslClientAuthenticationValidValues = SslClientAuth.VALUES.asScala.map(v => v.toString.toLowerCase(Locale.ROOT)).asJava.toArray(new Array[String](0)) - val SslPrincipalMappingRules = BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES - - /** ********* General Security configuration ***********/ - val ConnectionsMaxReauthMsDefault = 0L - val DefaultServerMaxMaxReceiveSize = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE - val DefaultPrincipalBuilder = classOf[DefaultKafkaPrincipalBuilder] - - /** ********* Sasl configuration ***********/ - val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM - val SaslEnabledMechanisms = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS - val SaslKerberosKinitCmd = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD - val SaslKerberosTicketRenewWindowFactor = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR - val SaslKerberosTicketRenewJitter = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER - val SaslKerberosMinTimeBeforeRelogin = SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN - val SaslKerberosPrincipalToLocalRules = BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES - val SaslLoginRefreshWindowFactor = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR - val SaslLoginRefreshWindowJitter = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER - val SaslLoginRefreshMinPeriodSeconds = SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS - val SaslLoginRefreshBufferSeconds = SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS - val SaslLoginRetryBackoffMaxMs = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS - val SaslLoginRetryBackoffMs = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS - val SaslOAuthBearerScopeClaimName = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SCOPE_CLAIM_NAME - val SaslOAuthBearerSubClaimName = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_SUB_CLAIM_NAME - val SaslOAuthBearerJwksEndpointRefreshMs = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS - val SaslOAuthBearerJwksEndpointRetryBackoffMaxMs = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS - val SaslOAuthBearerJwksEndpointRetryBackoffMs = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS - val SaslOAuthBearerClockSkewSeconds = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS - - /** ********* Delegation Token configuration ***********/ - val DelegationTokenMaxLifeTimeMsDefault = 7 * 24 * 60 * 60 * 1000L - val DelegationTokenExpiryTimeMsDefault = 24 * 60 * 60 * 1000L - val DelegationTokenExpiryCheckIntervalMsDefault = 1 * 60 * 60 * 1000L - - /** ********* Password encryption configuration for dynamic configs *********/ - val PasswordEncoderCipherAlgorithm = "AES/CBC/PKCS5Padding" - val PasswordEncoderKeyLength = 128 - val PasswordEncoderIterations = 4096 - - /** ********* Raft Quorum Configuration *********/ - val QuorumVoters = RaftConfig.DEFAULT_QUORUM_VOTERS - val QuorumElectionTimeoutMs = RaftConfig.DEFAULT_QUORUM_ELECTION_TIMEOUT_MS - val QuorumFetchTimeoutMs = RaftConfig.DEFAULT_QUORUM_FETCH_TIMEOUT_MS - val QuorumElectionBackoffMs = RaftConfig.DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS - val QuorumLingerMs = RaftConfig.DEFAULT_QUORUM_LINGER_MS - val QuorumRequestTimeoutMs = RaftConfig.DEFAULT_QUORUM_REQUEST_TIMEOUT_MS - val QuorumRetryBackoffMs = RaftConfig.DEFAULT_QUORUM_RETRY_BACKOFF_MS -} - object KafkaConfig { private val LogConfigPrefix = "log." @@ -758,18 +519,18 @@ object KafkaConfig { "If it is not set, the metadata log is placed in the first log directory from log.dirs." val MetadataSnapshotMaxNewRecordBytesDoc = "This is the maximum number of bytes in the log between the latest " + "snapshot and the high-watermark needed before generating a new snapshot. The default value is " + - s"${Defaults.MetadataSnapshotMaxNewRecordBytes}. To generate snapshots based on the time elapsed, see " + + s"${Defaults.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES}. To generate snapshots based on the time elapsed, see " + s"the $MetadataSnapshotMaxIntervalMsProp configuration. The Kafka node will generate a snapshot when " + "either the maximum time interval is reached or the maximum bytes limit is reached." val MetadataSnapshotMaxIntervalMsDoc = "This is the maximum number of milliseconds to wait to generate a snapshot " + "if there are committed records in the log that are not included in the latest snapshot. A value of zero disables " + - s"time based snapshot generation. The default value is ${Defaults.MetadataSnapshotMaxIntervalMs}. To generate " + + s"time based snapshot generation. The default value is ${Defaults.METADATA_SNAPSHOT_MAX_INTERVAL_MS}. To generate " + s"snapshots based on the number of metadata bytes, see the $MetadataSnapshotMaxNewRecordBytesProp " + "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "maximum bytes limit is reached." val MetadataMaxIdleIntervalMsDoc = "This configuration controls how often the active " + "controller should write no-op records to the metadata partition. If the value is 0, no-op records " + - s"are not appended to the metadata partition. The default value is ${Defaults.MetadataMaxIdleIntervalMs}" + s"are not appended to the metadata partition. The default value is ${Defaults.METADATA_MAX_IDLE_INTERVAL_MS}" val ControllerListenerNamesDoc = "A comma-separated list of the names of the listeners used by the controller. This is required " + "if running in KRaft mode. When communicating with the controller quorum, the broker will always use the first listener in this list.\n " + "Note: The ZooKeeper-based controller should not set this configuration." @@ -1204,11 +965,11 @@ object KafkaConfig { /** ********* Zookeeper Configuration ***********/ .define(ZkConnectProp, STRING, null, HIGH, ZkConnectDoc) - .define(ZkSessionTimeoutMsProp, INT, Defaults.ZkSessionTimeoutMs, HIGH, ZkSessionTimeoutMsDoc) + .define(ZkSessionTimeoutMsProp, INT, Defaults.ZK_SESSION_TIMEOUT_MS, HIGH, ZkSessionTimeoutMsDoc) .define(ZkConnectionTimeoutMsProp, INT, null, HIGH, ZkConnectionTimeoutMsDoc) - .define(ZkEnableSecureAclsProp, BOOLEAN, Defaults.ZkEnableSecureAcls, HIGH, ZkEnableSecureAclsDoc) - .define(ZkMaxInFlightRequestsProp, INT, Defaults.ZkMaxInFlightRequests, atLeast(1), HIGH, ZkMaxInFlightRequestsDoc) - .define(ZkSslClientEnableProp, BOOLEAN, Defaults.ZkSslClientEnable, MEDIUM, ZkSslClientEnableDoc) + .define(ZkEnableSecureAclsProp, BOOLEAN, Defaults.ZK_ENABLE_SECURE_ACLS, HIGH, ZkEnableSecureAclsDoc) + .define(ZkMaxInFlightRequestsProp, INT, Defaults.ZK_MAX_IN_FLIGHT_REQUESTS, atLeast(1), HIGH, ZkMaxInFlightRequestsDoc) + .define(ZkSslClientEnableProp, BOOLEAN, Defaults.ZK_SSL_CLIENT_ENABLE, MEDIUM, ZkSslClientEnableDoc) .define(ZkClientCnxnSocketProp, STRING, null, MEDIUM, ZkClientCnxnSocketDoc) .define(ZkSslKeyStoreLocationProp, STRING, null, MEDIUM, ZkSslKeyStoreLocationDoc) .define(ZkSslKeyStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslKeyStorePasswordDoc) @@ -1216,77 +977,77 @@ object KafkaConfig { .define(ZkSslTrustStoreLocationProp, STRING, null, MEDIUM, ZkSslTrustStoreLocationDoc) .define(ZkSslTrustStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslTrustStorePasswordDoc) .define(ZkSslTrustStoreTypeProp, STRING, null, MEDIUM, ZkSslTrustStoreTypeDoc) - .define(ZkSslProtocolProp, STRING, Defaults.ZkSslProtocol, LOW, ZkSslProtocolDoc) + .define(ZkSslProtocolProp, STRING, Defaults.ZK_SSL_PROTOCOL, LOW, ZkSslProtocolDoc) .define(ZkSslEnabledProtocolsProp, LIST, null, LOW, ZkSslEnabledProtocolsDoc) .define(ZkSslCipherSuitesProp, LIST, null, LOW, ZkSslCipherSuitesDoc) - .define(ZkSslEndpointIdentificationAlgorithmProp, STRING, Defaults.ZkSslEndpointIdentificationAlgorithm, LOW, ZkSslEndpointIdentificationAlgorithmDoc) - .define(ZkSslCrlEnableProp, BOOLEAN, Defaults.ZkSslCrlEnable, LOW, ZkSslCrlEnableDoc) - .define(ZkSslOcspEnableProp, BOOLEAN, Defaults.ZkSslOcspEnable, LOW, ZkSslOcspEnableDoc) + .define(ZkSslEndpointIdentificationAlgorithmProp, STRING, Defaults.ZK_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, ZkSslEndpointIdentificationAlgorithmDoc) + .define(ZkSslCrlEnableProp, BOOLEAN, Defaults.ZK_SSL_CRL_ENABLE, LOW, ZkSslCrlEnableDoc) + .define(ZkSslOcspEnableProp, BOOLEAN, Defaults.ZK_SSL_OCSP_ENABLE, LOW, ZkSslOcspEnableDoc) /** ********* General Configuration ***********/ - .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BrokerIdGenerationEnable, MEDIUM, BrokerIdGenerationEnableDoc) - .define(MaxReservedBrokerIdProp, INT, Defaults.MaxReservedBrokerId, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) - .define(BrokerIdProp, INT, Defaults.BrokerId, HIGH, BrokerIdDoc) + .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.NumNetworkThreads, atLeast(1), HIGH, NumNetworkThreadsDoc) - .define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) + .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.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) - .define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) - .define(QueuedMaxBytesProp, LONG, Defaults.QueuedMaxRequestBytes, MEDIUM, QueuedMaxRequestBytesDoc) - .define(RequestTimeoutMsProp, INT, Defaults.RequestTimeoutMs, HIGH, RequestTimeoutMsDoc) - .define(ConnectionSetupTimeoutMsProp, LONG, Defaults.ConnectionSetupTimeoutMs, MEDIUM, ConnectionSetupTimeoutMsDoc) - .define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.ConnectionSetupTimeoutMaxMs, MEDIUM, ConnectionSetupTimeoutMaxMsDoc) + .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) /* * KRaft mode configs. */ - .define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.MetadataSnapshotMaxNewRecordBytes, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc) - .define(MetadataSnapshotMaxIntervalMsProp, LONG, Defaults.MetadataSnapshotMaxIntervalMs, atLeast(0), HIGH, MetadataSnapshotMaxIntervalMsDoc) + .define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc) + .define(MetadataSnapshotMaxIntervalMsProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_INTERVAL_MS, atLeast(0), HIGH, MetadataSnapshotMaxIntervalMsDoc) .define(ProcessRolesProp, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, ProcessRolesDoc) - .define(NodeIdProp, INT, Defaults.EmptyNodeId, null, HIGH, NodeIdDoc) - .define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.InitialBrokerRegistrationTimeoutMs, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc) - .define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BrokerHeartbeatIntervalMs, null, MEDIUM, BrokerHeartbeatIntervalMsDoc) - .define(BrokerSessionTimeoutMsProp, INT, Defaults.BrokerSessionTimeoutMs, null, MEDIUM, BrokerSessionTimeoutMsDoc) + .define(NodeIdProp, INT, Defaults.EMPTY_NODE_ID, null, HIGH, NodeIdDoc) + .define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc) + .define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BROKER_HEARTBEAT_INTERVAL_MS, null, MEDIUM, BrokerHeartbeatIntervalMsDoc) + .define(BrokerSessionTimeoutMsProp, INT, Defaults.BROKER_SESSION_TIMEOUT_MS, null, MEDIUM, BrokerSessionTimeoutMsDoc) .define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc) .define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc) .define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc) .define(MetadataLogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc) .defineInternal(MetadataLogSegmentMinBytesProp, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentMinBytesDoc) .define(MetadataLogSegmentMillisProp, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, MetadataLogSegmentMillisDoc) - .define(MetadataMaxRetentionBytesProp, LONG, Defaults.MetadataMaxRetentionBytes, null, HIGH, MetadataMaxRetentionBytesDoc) + .define(MetadataMaxRetentionBytesProp, LONG, Defaults.METADATA_MAX_RETENTION_BYTES, null, HIGH, MetadataMaxRetentionBytesDoc) .define(MetadataMaxRetentionMillisProp, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, MetadataMaxRetentionMillisDoc) - .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) - .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) + .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.METADATA_MAX_IDLE_INTERVAL_MS, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) + .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas") /************* Authorizer Configuration ***********/ - .define(AuthorizerClassNameProp, STRING, Defaults.AuthorizerClassName, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) + .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) .define(EarlyStartListenersProp, STRING, null, HIGH, EarlyStartListenersDoc) /** ********* Socket Server Configuration ***********/ - .define(ListenersProp, STRING, Defaults.Listeners, HIGH, ListenersDoc) + .define(ListenersProp, STRING, Defaults.LISTENERS, HIGH, ListenersDoc) .define(AdvertisedListenersProp, STRING, null, HIGH, AdvertisedListenersDoc) - .define(ListenerSecurityProtocolMapProp, STRING, Defaults.ListenerSecurityProtocolMap, LOW, ListenerSecurityProtocolMapDoc) + .define(ListenerSecurityProtocolMapProp, STRING, Defaults.LISTENER_SECURITY_PROTOCOL_MAP, LOW, ListenerSecurityProtocolMapDoc) .define(ControlPlaneListenerNameProp, STRING, null, HIGH, controlPlaneListenerNameDoc) - .define(SocketSendBufferBytesProp, INT, Defaults.SocketSendBufferBytes, HIGH, SocketSendBufferBytesDoc) - .define(SocketReceiveBufferBytesProp, INT, Defaults.SocketReceiveBufferBytes, HIGH, SocketReceiveBufferBytesDoc) - .define(SocketRequestMaxBytesProp, INT, Defaults.SocketRequestMaxBytes, atLeast(1), HIGH, SocketRequestMaxBytesDoc) - .define(SocketListenBacklogSizeProp, INT, Defaults.SocketListenBacklogSize, atLeast(1), MEDIUM, SocketListenBacklogSizeDoc) - .define(MaxConnectionsPerIpProp, INT, Defaults.MaxConnectionsPerIp, atLeast(0), MEDIUM, MaxConnectionsPerIpDoc) - .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MaxConnectionsPerIpOverrides, MEDIUM, MaxConnectionsPerIpOverridesDoc) - .define(MaxConnectionsProp, INT, Defaults.MaxConnections, atLeast(0), MEDIUM, MaxConnectionsDoc) - .define(MaxConnectionCreationRateProp, INT, Defaults.MaxConnectionCreationRate, atLeast(0), MEDIUM, MaxConnectionCreationRateDoc) - .define(ConnectionsMaxIdleMsProp, LONG, Defaults.ConnectionsMaxIdleMs, MEDIUM, ConnectionsMaxIdleMsDoc) - .define(FailedAuthenticationDelayMsProp, INT, Defaults.FailedAuthenticationDelayMs, atLeast(0), LOW, FailedAuthenticationDelayMsDoc) + .define(SocketSendBufferBytesProp, INT, Defaults.SOCKET_SEND_BUFFER_BYTES, HIGH, SocketSendBufferBytesDoc) + .define(SocketReceiveBufferBytesProp, INT, Defaults.SOCKET_RECEIVE_BUFFER_BYTES, HIGH, SocketReceiveBufferBytesDoc) + .define(SocketRequestMaxBytesProp, INT, Defaults.SOCKET_REQUEST_MAX_BYTES, atLeast(1), HIGH, SocketRequestMaxBytesDoc) + .define(SocketListenBacklogSizeProp, INT, Defaults.SOCKET_LISTEN_BACKLOG_SIZE, atLeast(1), MEDIUM, SocketListenBacklogSizeDoc) + .define(MaxConnectionsPerIpProp, INT, Defaults.MAX_CONNECTIONS_PER_IP, atLeast(0), MEDIUM, MaxConnectionsPerIpDoc) + .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MAX_CONNECTIONS_PER_IP_OVERRIDES, MEDIUM, MaxConnectionsPerIpOverridesDoc) + .define(MaxConnectionsProp, INT, Defaults.MAX_CONNECTIONS, atLeast(0), MEDIUM, MaxConnectionsDoc) + .define(MaxConnectionCreationRateProp, INT, Defaults.MAX_CONNECTION_CREATION_RATE, atLeast(0), MEDIUM, MaxConnectionCreationRateDoc) + .define(ConnectionsMaxIdleMsProp, LONG, Defaults.CONNECTIONS_MAX_IDLE_MS, MEDIUM, ConnectionsMaxIdleMsDoc) + .define(FailedAuthenticationDelayMsProp, INT, Defaults.FAILED_AUTHENTICATION_DELAY_MS, atLeast(0), LOW, FailedAuthenticationDelayMsDoc) /************ Rack Configuration ******************/ .define(RackProp, STRING, null, MEDIUM, RackDoc) /** ********* Log Configuration ***********/ - .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) - .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) + .define(NumPartitionsProp, INT, Defaults.NUM_PARTITIONS, atLeast(1), MEDIUM, NumPartitionsDoc) + .define(LogDirProp, STRING, Defaults.LOG_DIR, HIGH, LogDirDoc) .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc) .define(LogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) @@ -1301,16 +1062,16 @@ object KafkaConfig { .define(LogRetentionTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, LogRetentionTimeHoursDoc) .define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc) - .define(LogCleanupIntervalMsProp, LONG, Defaults.LogCleanupIntervalMs, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) + .define(LogCleanupIntervalMsProp, LONG, Defaults.LOG_CLEANUP_INTERVAL_MS, atLeast(1), MEDIUM, LogCleanupIntervalMsDoc) .define(LogCleanupPolicyProp, LIST, LogConfig.DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT, TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, LogCleanupPolicyDoc) - .define(LogCleanerThreadsProp, INT, Defaults.LogCleanerThreads, atLeast(0), MEDIUM, LogCleanerThreadsDoc) - .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LogCleanerIoMaxBytesPerSecond, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) - .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LogCleanerDedupeBufferSize, MEDIUM, LogCleanerDedupeBufferSizeDoc) - .define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) - .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) - .define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) + .define(LogCleanerThreadsProp, INT, Defaults.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, LogCleanerThreadsDoc) + .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) + .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, LogCleanerDedupeBufferSizeDoc) + .define(LogCleanerIoBufferSizeProp, INT, Defaults.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) + .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) + .define(LogCleanerBackoffMsProp, LONG, Defaults.LOG_CLEANER_BACKOFF_MS, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) .define(LogCleanerMinCleanRatioProp, DOUBLE, LogConfig.DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM, LogCleanerMinCleanRatioDoc) - .define(LogCleanerEnableProp, BOOLEAN, Defaults.LogCleanerEnable, MEDIUM, LogCleanerEnableDoc) + .define(LogCleanerEnableProp, BOOLEAN, Defaults.LOG_CLEANER_ENABLE, MEDIUM, LogCleanerEnableDoc) .define(LogCleanerDeleteRetentionMsProp, LONG, LogConfig.DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM, LogCleanerDeleteRetentionMsDoc) .define(LogCleanerMinCompactionLagMsProp, LONG, LogConfig.DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM, LogCleanerMinCompactionLagMsDoc) .define(LogCleanerMaxCompactionLagMsProp, LONG, LogConfig.DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM, LogCleanerMaxCompactionLagMsDoc) @@ -1320,11 +1081,11 @@ object KafkaConfig { .define(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc) .define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc) .define(LogFlushIntervalMsProp, LONG, null, HIGH, LogFlushIntervalMsDoc) - .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) - .define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushStartOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc) + .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) + .define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc) .define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc) - .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) - .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) + .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NUM_RECOVERY_THREADS_PER_DATA_DIR, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) + .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AUTO_CREATE_TOPICS_ENABLE, HIGH, AutoCreateTopicsEnableDoc) .define(MinInSyncReplicasProp, INT, LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), HIGH, MinInSyncReplicasDoc) .define(LogMessageFormatVersionProp, STRING, LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc) .define(LogMessageTimestampTypeProp, STRING, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_TYPE, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) @@ -1336,205 +1097,205 @@ object KafkaConfig { .define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc) /** ********* Replication configuration ***********/ - .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) - .define(DefaultReplicationFactorProp, INT, Defaults.DefaultReplicationFactor, MEDIUM, DefaultReplicationFactorDoc) - .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.ReplicaLagTimeMaxMs, HIGH, ReplicaLagTimeMaxMsDoc) - .define(ReplicaSocketTimeoutMsProp, INT, Defaults.ReplicaSocketTimeoutMs, HIGH, ReplicaSocketTimeoutMsDoc) - .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.ReplicaSocketReceiveBufferBytes, HIGH, ReplicaSocketReceiveBufferBytesDoc) - .define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, atLeast(0), MEDIUM, ReplicaFetchMaxBytesDoc) - .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) - .define(ReplicaFetchBackoffMsProp, INT, Defaults.ReplicaFetchBackoffMs, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) - .define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) - .define(ReplicaFetchResponseMaxBytesProp, INT, Defaults.ReplicaFetchResponseMaxBytes, atLeast(0), MEDIUM, ReplicaFetchResponseMaxBytesDoc) - .define(NumReplicaFetchersProp, INT, Defaults.NumReplicaFetchers, HIGH, NumReplicaFetchersDoc) - .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.ReplicaHighWatermarkCheckpointIntervalMs, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) - .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FetchPurgatoryPurgeIntervalRequests, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) - .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) - .define(DeleteRecordsPurgatoryPurgeIntervalRequestsProp, INT, Defaults.DeleteRecordsPurgatoryPurgeIntervalRequests, MEDIUM, DeleteRecordsPurgatoryPurgeIntervalRequestsDoc) - .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) - .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) - .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc) + .define(ControllerSocketTimeoutMsProp, INT, Defaults.CONTROLLER_SOCKET_TIMEOUT_MS, MEDIUM, ControllerSocketTimeoutMsDoc) + .define(DefaultReplicationFactorProp, INT, Defaults.REPLICATION_FACTOR, MEDIUM, DefaultReplicationFactorDoc) + .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.REPLICA_LAG_TIME_MAX_MS, HIGH, ReplicaLagTimeMaxMsDoc) + .define(ReplicaSocketTimeoutMsProp, INT, Defaults.REPLICA_SOCKET_TIMEOUT_MS, HIGH, ReplicaSocketTimeoutMsDoc) + .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.REPLICA_SOCKET_RECEIVE_BUFFER_BYTES, HIGH, ReplicaSocketReceiveBufferBytesDoc) + .define(ReplicaFetchMaxBytesProp, INT, Defaults.REPLICA_FETCH_MAX_BYTES, atLeast(0), MEDIUM, ReplicaFetchMaxBytesDoc) + .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.REPLICA_FETCH_WAIT_MAX_MS, HIGH, ReplicaFetchWaitMaxMsDoc) + .define(ReplicaFetchBackoffMsProp, INT, Defaults.REPLICA_FETCH_BACKOFF_MS, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) + .define(ReplicaFetchMinBytesProp, INT, Defaults.REPLICA_FETCH_MIN_BYTES, HIGH, ReplicaFetchMinBytesDoc) + .define(ReplicaFetchResponseMaxBytesProp, INT, Defaults.REPLICA_FETCH_RESPONSE_MAX_BYTES, atLeast(0), MEDIUM, ReplicaFetchResponseMaxBytesDoc) + .define(NumReplicaFetchersProp, INT, Defaults.NUM_REPLICA_FETCHERS, HIGH, NumReplicaFetchersDoc) + .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) + .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) + .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) + .define(DeleteRecordsPurgatoryPurgeIntervalRequestsProp, INT, Defaults.DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, DeleteRecordsPurgatoryPurgeIntervalRequestsDoc) + .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AUTO_LEADER_REBALANCE_ENABLE, HIGH, AutoLeaderRebalanceEnableDoc) + .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE, HIGH, LeaderImbalancePerBrokerPercentageDoc) + .define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .define(UncleanLeaderElectionEnableProp, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, UncleanLeaderElectionEnableDoc) - .define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, InterBrokerSecurityProtocolDoc) - .define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc) + .define(InterBrokerSecurityProtocolProp, STRING, Defaults.INTER_BROKER_SECURITY_PROTOCOL, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, InterBrokerSecurityProtocolDoc) + .define(InterBrokerProtocolVersionProp, STRING, Defaults.INTER_BROKER_PROTOCOL_VERSION, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc) .define(InterBrokerListenerNameProp, STRING, null, MEDIUM, InterBrokerListenerNameDoc) .define(ReplicaSelectorClassProp, STRING, null, MEDIUM, ReplicaSelectorClassDoc) /** ********* Controlled shutdown configuration ***********/ - .define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) - .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) - .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) + .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) /** ********* Group coordinator configuration ***********/ - .define(GroupMinSessionTimeoutMsProp, INT, Defaults.GroupMinSessionTimeoutMs, MEDIUM, GroupMinSessionTimeoutMsDoc) - .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GroupMaxSessionTimeoutMs, MEDIUM, GroupMaxSessionTimeoutMsDoc) - .define(GroupInitialRebalanceDelayMsProp, INT, Defaults.GroupInitialRebalanceDelayMs, MEDIUM, GroupInitialRebalanceDelayMsDoc) - .define(GroupMaxSizeProp, INT, Defaults.GroupMaxSize, atLeast(1), MEDIUM, GroupMaxSizeDoc) + .define(GroupMinSessionTimeoutMsProp, INT, Defaults.GROUP_MIN_SESSION_TIMEOUT_MS, MEDIUM, GroupMinSessionTimeoutMsDoc) + .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GROUP_MAX_SESSION_TIMEOUT_MS, MEDIUM, GroupMaxSessionTimeoutMsDoc) + .define(GroupInitialRebalanceDelayMsProp, INT, Defaults.GROUP_INITIAL_REBALANCE_DELAY_MS, MEDIUM, GroupInitialRebalanceDelayMsDoc) + .define(GroupMaxSizeProp, INT, Defaults.GROUP_MAX_SIZE, atLeast(1), MEDIUM, GroupMaxSizeDoc) /** New group coordinator configs */ - .define(GroupCoordinatorRebalanceProtocolsProp, LIST, Defaults.GroupCoordinatorRebalanceProtocols, + .define(GroupCoordinatorRebalanceProtocolsProp, LIST, Defaults.GROUP_COORDINATOR_REBALANCE_PROTOCOLS, ConfigDef.ValidList.in(Utils.enumOptions(classOf[GroupType]):_*), MEDIUM, GroupCoordinatorRebalanceProtocolsDoc) - .define(GroupCoordinatorNumThreadsProp, INT, Defaults.GroupCoordinatorNumThreads, atLeast(1), MEDIUM, GroupCoordinatorNumThreadsDoc) + .define(GroupCoordinatorNumThreadsProp, INT, Defaults.GROUP_COORDINATOR_NUM_THREADS, atLeast(1), MEDIUM, GroupCoordinatorNumThreadsDoc) // Internal configuration used by integration and system tests. - .defineInternal(NewGroupCoordinatorEnableProp, BOOLEAN, Defaults.NewGroupCoordinatorEnable, null, MEDIUM, NewGroupCoordinatorEnableDoc) + .defineInternal(NewGroupCoordinatorEnableProp, BOOLEAN, Defaults.NEW_GROUP_COORDINATOR_ENABLE, null, MEDIUM, NewGroupCoordinatorEnableDoc) /** Consumer groups configs */ - .define(ConsumerGroupSessionTimeoutMsProp, INT, Defaults.ConsumerGroupSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupSessionTimeoutMsDoc) - .define(ConsumerGroupMinSessionTimeoutMsProp, INT, Defaults.ConsumerGroupMinSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupMinSessionTimeoutMsDoc) - .define(ConsumerGroupMaxSessionTimeoutMsProp, INT, Defaults.ConsumerGroupMaxSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupMaxSessionTimeoutMsDoc) - .define(ConsumerGroupHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupHeartbeatIntervalMsDoc) - .define(ConsumerGroupMinHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupMinHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupMinHeartbeatIntervalMsDoc) - .define(ConsumerGroupMaxHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupMaxHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupMaxHeartbeatIntervalMsDoc) - .define(ConsumerGroupMaxSizeProp, INT, Defaults.ConsumerGroupMaxSize, atLeast(1), MEDIUM, ConsumerGroupMaxSizeDoc) - .define(ConsumerGroupAssignorsProp, LIST, Defaults.ConsumerGroupAssignors, null, MEDIUM, ConsumerGroupAssignorsDoc) + .define(ConsumerGroupSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupSessionTimeoutMsDoc) + .define(ConsumerGroupMinSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupMinSessionTimeoutMsDoc) + .define(ConsumerGroupMaxSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupMaxSessionTimeoutMsDoc) + .define(ConsumerGroupHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupHeartbeatIntervalMsDoc) + .define(ConsumerGroupMinHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupMinHeartbeatIntervalMsDoc) + .define(ConsumerGroupMaxHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupMaxHeartbeatIntervalMsDoc) + .define(ConsumerGroupMaxSizeProp, INT, Defaults.CONSUMER_GROUP_MAX_SIZE, atLeast(1), MEDIUM, ConsumerGroupMaxSizeDoc) + .define(ConsumerGroupAssignorsProp, LIST, Defaults.CONSUMER_GROUP_ASSIGNORS, null, MEDIUM, ConsumerGroupAssignorsDoc) /** ********* Offset management configuration ***********/ - .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) - .define(OffsetsLoadBufferSizeProp, INT, Defaults.OffsetsLoadBufferSize, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) - .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OffsetsTopicReplicationFactor, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc) - .define(OffsetsTopicPartitionsProp, INT, Defaults.OffsetsTopicPartitions, atLeast(1), HIGH, OffsetsTopicPartitionsDoc) - .define(OffsetsTopicSegmentBytesProp, INT, Defaults.OffsetsTopicSegmentBytes, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc) - .define(OffsetsTopicCompressionCodecProp, INT, Defaults.OffsetsTopicCompressionCodec, HIGH, OffsetsTopicCompressionCodecDoc) - .define(OffsetsRetentionMinutesProp, INT, Defaults.OffsetsRetentionMinutes, atLeast(1), HIGH, OffsetsRetentionMinutesDoc) - .define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OffsetsRetentionCheckIntervalMs, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc) - .define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) - .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) - .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) + .define(OffsetMetadataMaxSizeProp, INT, Defaults.OFFSET_METADATA_MAX_SIZE, HIGH, OffsetMetadataMaxSizeDoc) + .define(OffsetsLoadBufferSizeProp, INT, Defaults.OFFSETS_LOAD_BUFFER_SIZE, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) + .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OFFSETS_TOPIC_REPLICATION_FACTOR, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc) + .define(OffsetsTopicPartitionsProp, INT, Defaults.OFFSETS_TOPIC_PARTITIONS, atLeast(1), HIGH, OffsetsTopicPartitionsDoc) + .define(OffsetsTopicSegmentBytesProp, INT, Defaults.OFFSETS_TOPIC_SEGMENT_BYTES, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc) + .define(OffsetsTopicCompressionCodecProp, INT, Defaults.OFFSETS_TOPIC_COMPRESSION_CODEC, HIGH, OffsetsTopicCompressionCodecDoc) + .define(OffsetsRetentionMinutesProp, INT, Defaults.OFFSETS_RETENTION_MINUTES, atLeast(1), HIGH, OffsetsRetentionMinutesDoc) + .define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OFFSETS_RETENTION_CHECK_INTERVAL_MS, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc) + .define(OffsetCommitTimeoutMsProp, INT, Defaults.OFFSET_COMMIT_TIMEOUT_MS, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) + .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OFFSET_COMMIT_REQUIRED_ACKS, HIGH, OffsetCommitRequiredAcksDoc) + .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc) .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) /** ********* Transaction management configuration ***********/ - .define(TransactionalIdExpirationMsProp, INT, Defaults.TransactionalIdExpirationMs, atLeast(1), HIGH, TransactionalIdExpirationMsDoc) - .define(TransactionsMaxTimeoutMsProp, INT, Defaults.TransactionsMaxTimeoutMs, atLeast(1), HIGH, TransactionsMaxTimeoutMsDoc) - .define(TransactionsTopicMinISRProp, INT, Defaults.TransactionsTopicMinISR, atLeast(1), HIGH, TransactionsTopicMinISRDoc) - .define(TransactionsLoadBufferSizeProp, INT, Defaults.TransactionsLoadBufferSize, atLeast(1), HIGH, TransactionsLoadBufferSizeDoc) - .define(TransactionsTopicReplicationFactorProp, SHORT, Defaults.TransactionsTopicReplicationFactor, atLeast(1), HIGH, TransactionsTopicReplicationFactorDoc) - .define(TransactionsTopicPartitionsProp, INT, Defaults.TransactionsTopicPartitions, atLeast(1), HIGH, TransactionsTopicPartitionsDoc) - .define(TransactionsTopicSegmentBytesProp, INT, Defaults.TransactionsTopicSegmentBytes, atLeast(1), HIGH, TransactionsTopicSegmentBytesDoc) - .define(TransactionsAbortTimedOutTransactionCleanupIntervalMsProp, INT, Defaults.TransactionsAbortTimedOutTransactionsCleanupIntervalMS, atLeast(1), LOW, TransactionsAbortTimedOutTransactionsIntervalMsDoc) - .define(TransactionsRemoveExpiredTransactionalIdCleanupIntervalMsProp, INT, Defaults.TransactionsRemoveExpiredTransactionsCleanupIntervalMS, atLeast(1), LOW, TransactionsRemoveExpiredTransactionsIntervalMsDoc) + .define(TransactionalIdExpirationMsProp, INT, Defaults.TRANSACTIONAL_ID_EXPIRATION_MS, atLeast(1), HIGH, TransactionalIdExpirationMsDoc) + .define(TransactionsMaxTimeoutMsProp, INT, Defaults.TRANSACTIONS_MAX_TIMEOUT_MS, atLeast(1), HIGH, TransactionsMaxTimeoutMsDoc) + .define(TransactionsTopicMinISRProp, INT, Defaults.TRANSACTIONS_TOPIC_MIN_ISR, atLeast(1), HIGH, TransactionsTopicMinISRDoc) + .define(TransactionsLoadBufferSizeProp, INT, Defaults.TRANSACTIONS_LOAD_BUFFER_SIZE, atLeast(1), HIGH, TransactionsLoadBufferSizeDoc) + .define(TransactionsTopicReplicationFactorProp, SHORT, Defaults.TRANSACTIONS_TOPIC_REPLICATION_FACTOR, atLeast(1), HIGH, TransactionsTopicReplicationFactorDoc) + .define(TransactionsTopicPartitionsProp, INT, Defaults.TRANSACTIONS_TOPIC_PARTITIONS, atLeast(1), HIGH, TransactionsTopicPartitionsDoc) + .define(TransactionsTopicSegmentBytesProp, INT, Defaults.TRANSACTIONS_TOPIC_SEGMENT_BYTES, atLeast(1), HIGH, TransactionsTopicSegmentBytesDoc) + .define(TransactionsAbortTimedOutTransactionCleanupIntervalMsProp, INT, Defaults.TRANSACTIONS_ABORT_TIMED_OUT_CLEANUP_INTERVAL_MS, atLeast(1), LOW, TransactionsAbortTimedOutTransactionsIntervalMsDoc) + .define(TransactionsRemoveExpiredTransactionalIdCleanupIntervalMsProp, INT, Defaults.TRANSACTIONS_REMOVE_EXPIRED_CLEANUP_INTERVAL_MS, atLeast(1), LOW, TransactionsRemoveExpiredTransactionsIntervalMsDoc) - .define(TransactionPartitionVerificationEnableProp, BOOLEAN, Defaults.TransactionPartitionVerificationEnable, LOW, TransactionPartitionVerificationEnableDoc) + .define(TransactionPartitionVerificationEnableProp, BOOLEAN, Defaults.TRANSACTION_PARTITION_VERIFICATION_ENABLE, LOW, TransactionPartitionVerificationEnableDoc) - .define(ProducerIdExpirationMsProp, INT, Defaults.ProducerIdExpirationMs, atLeast(1), LOW, ProducerIdExpirationMsDoc) + .define(ProducerIdExpirationMsProp, INT, Defaults.PRODUCER_ID_EXPIRATION_MS, atLeast(1), LOW, ProducerIdExpirationMsDoc) // Configuration for testing only as default value should be sufficient for typical usage - .defineInternal(ProducerIdExpirationCheckIntervalMsProp, INT, Defaults.ProducerIdExpirationCheckIntervalMs, atLeast(1), LOW, ProducerIdExpirationCheckIntervalMsDoc) + .defineInternal(ProducerIdExpirationCheckIntervalMsProp, INT, Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, atLeast(1), LOW, ProducerIdExpirationCheckIntervalMsDoc) /** ********* Fetch Configuration **************/ - .define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MaxIncrementalFetchSessionCacheSlots, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc) - .define(FetchMaxBytes, INT, Defaults.FetchMaxBytes, atLeast(1024), MEDIUM, FetchMaxBytesDoc) + .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) /** ********* Kafka Metrics Configuration ***********/ - .define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) - .define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) - .define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) - .define(MetricRecordingLevelProp, STRING, Defaults.MetricRecordingLevel, LOW, MetricRecordingLevelDoc) - .define(AutoIncludeJmxReporterProp, BOOLEAN, Defaults.AutoIncludeJmxReporter, LOW, AutoIncludeJmxReporterDoc) + .define(MetricNumSamplesProp, INT, Defaults.METRIC_NUM_SAMPLES, atLeast(1), LOW, MetricNumSamplesDoc) + .define(MetricSampleWindowMsProp, LONG, Defaults.METRIC_SAMPLE_WINDOW_MS, atLeast(1), LOW, MetricSampleWindowMsDoc) + .define(MetricReporterClassesProp, LIST, Defaults.METRIC_REPORTER_CLASSES, LOW, MetricReporterClassesDoc) + .define(MetricRecordingLevelProp, STRING, Defaults.METRIC_RECORDING_LEVEL, LOW, MetricRecordingLevelDoc) + .define(AutoIncludeJmxReporterProp, BOOLEAN, Defaults.AUTO_INCLUDE_JMX_REPORTER, LOW, AutoIncludeJmxReporterDoc) /** ********* Kafka Yammer Metrics Reporter Configuration for docs ***********/ - .define(KafkaMetricsReporterClassesProp, LIST, Defaults.KafkaMetricReporterClasses, LOW, KafkaMetricsReporterClassesDoc) - .define(KafkaMetricsPollingIntervalSecondsProp, INT, Defaults.KafkaMetricsPollingIntervalSeconds, atLeast(1), LOW, KafkaMetricsPollingIntervalSecondsDoc) + .define(KafkaMetricsReporterClassesProp, LIST, Defaults.KAFKA_METRIC_REPORTER_CLASSES, LOW, KafkaMetricsReporterClassesDoc) + .define(KafkaMetricsPollingIntervalSecondsProp, INT, Defaults.KAFKA_METRICS_POLLING_INTERVAL_SECONDS, atLeast(1), LOW, KafkaMetricsPollingIntervalSecondsDoc) /** ********* Kafka Client Telemetry Metrics Configuration ***********/ - .define(ClientTelemetryMaxBytesProp, INT, Defaults.ClientTelemetryMaxBytes, atLeast(1), LOW, ClientTelemetryMaxBytesDoc) + .define(ClientTelemetryMaxBytesProp, INT, Defaults.CLIENT_TELEMETRY_MAX_BYTES, atLeast(1), LOW, ClientTelemetryMaxBytesDoc) /** ********* Quota configuration ***********/ - .define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) - .define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc) - .define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NumAlterLogDirsReplicationQuotaSamples, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc) - .define(NumControllerQuotaSamplesProp, INT, Defaults.NumControllerQuotaSamples, atLeast(1), LOW, NumControllerQuotaSamplesDoc) - .define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) - .define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc) - .define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.AlterLogDirsReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc) - .define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.ControllerQuotaWindowSizeSeconds, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc) + .define(NumQuotaSamplesProp, INT, Defaults.NUM_QUOTA_SAMPLES, atLeast(1), LOW, NumQuotaSamplesDoc) + .define(NumReplicationQuotaSamplesProp, INT, Defaults.NUM_REPLICATION_QUOTA_SAMPLES, atLeast(1), LOW, NumReplicationQuotaSamplesDoc) + .define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc) + .define(NumControllerQuotaSamplesProp, INT, Defaults.NUM_CONTROLLER_QUOTA_SAMPLES, atLeast(1), LOW, NumControllerQuotaSamplesDoc) + .define(QuotaWindowSizeSecondsProp, INT, Defaults.QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) + .define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.REPLICATION_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc) + .define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc) + .define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc) .define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc) /** ********* General Security Configuration ****************/ - .define(ConnectionsMaxReauthMsProp, LONG, Defaults.ConnectionsMaxReauthMsDefault, MEDIUM, ConnectionsMaxReauthMsDoc) - .define(SaslServerMaxReceiveSizeProp, INT, Defaults.DefaultServerMaxMaxReceiveSize, MEDIUM, SaslServerMaxReceiveSizeDoc) + .define(ConnectionsMaxReauthMsProp, LONG, Defaults.CONNECTIONS_MAX_REAUTH_MS, MEDIUM, ConnectionsMaxReauthMsDoc) + .define(SaslServerMaxReceiveSizeProp, INT, Defaults.SERVER_MAX_RECEIVE_SIZE, MEDIUM, SaslServerMaxReceiveSizeDoc) .define(securityProviderClassProp, STRING, null, LOW, securityProviderClassDoc) /** ********* SSL Configuration ****************/ - .define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalBuilder, MEDIUM, PrincipalBuilderClassDoc) - .define(SslProtocolProp, STRING, Defaults.SslProtocol, MEDIUM, SslProtocolDoc) + .define(PrincipalBuilderClassProp, CLASS, Defaults.PRINCIPAL_BUILDER, MEDIUM, PrincipalBuilderClassDoc) + .define(SslProtocolProp, STRING, Defaults.SSL_PROTOCOL, MEDIUM, SslProtocolDoc) .define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc) - .define(SslEnabledProtocolsProp, LIST, Defaults.SslEnabledProtocols, MEDIUM, SslEnabledProtocolsDoc) - .define(SslKeystoreTypeProp, STRING, Defaults.SslKeystoreType, MEDIUM, SslKeystoreTypeDoc) + .define(SslEnabledProtocolsProp, LIST, Defaults.SSL_ENABLED_PROTOCOLS, MEDIUM, SslEnabledProtocolsDoc) + .define(SslKeystoreTypeProp, STRING, Defaults.SSL_KEYSTORE_TYPE, MEDIUM, SslKeystoreTypeDoc) .define(SslKeystoreLocationProp, STRING, null, MEDIUM, SslKeystoreLocationDoc) .define(SslKeystorePasswordProp, PASSWORD, null, MEDIUM, SslKeystorePasswordDoc) .define(SslKeyPasswordProp, PASSWORD, null, MEDIUM, SslKeyPasswordDoc) .define(SslKeystoreKeyProp, PASSWORD, null, MEDIUM, SslKeystoreKeyDoc) .define(SslKeystoreCertificateChainProp, PASSWORD, null, MEDIUM, SslKeystoreCertificateChainDoc) - .define(SslTruststoreTypeProp, STRING, Defaults.SslTruststoreType, MEDIUM, SslTruststoreTypeDoc) + .define(SslTruststoreTypeProp, STRING, Defaults.SSL_TRUSTSTORE_TYPE, MEDIUM, SslTruststoreTypeDoc) .define(SslTruststoreLocationProp, STRING, null, MEDIUM, SslTruststoreLocationDoc) .define(SslTruststorePasswordProp, PASSWORD, null, MEDIUM, SslTruststorePasswordDoc) .define(SslTruststoreCertificatesProp, PASSWORD, null, MEDIUM, SslTruststoreCertificatesDoc) - .define(SslKeyManagerAlgorithmProp, STRING, Defaults.SslKeyManagerAlgorithm, MEDIUM, SslKeyManagerAlgorithmDoc) - .define(SslTrustManagerAlgorithmProp, STRING, Defaults.SslTrustManagerAlgorithm, MEDIUM, SslTrustManagerAlgorithmDoc) - .define(SslEndpointIdentificationAlgorithmProp, STRING, Defaults.SslEndpointIdentificationAlgorithm, LOW, SslEndpointIdentificationAlgorithmDoc) + .define(SslKeyManagerAlgorithmProp, STRING, Defaults.SSL_KEY_MANAGER_ALGORITHM, MEDIUM, SslKeyManagerAlgorithmDoc) + .define(SslTrustManagerAlgorithmProp, STRING, Defaults.SSL_TRUST_MANAGER_ALGORITHM, MEDIUM, SslTrustManagerAlgorithmDoc) + .define(SslEndpointIdentificationAlgorithmProp, STRING, Defaults.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, SslEndpointIdentificationAlgorithmDoc) .define(SslSecureRandomImplementationProp, STRING, null, LOW, SslSecureRandomImplementationDoc) - .define(SslClientAuthProp, STRING, Defaults.SslClientAuthentication, in(Defaults.SslClientAuthenticationValidValues:_*), MEDIUM, SslClientAuthDoc) + .define(SslClientAuthProp, STRING, Defaults.SSL_CLIENT_AUTHENTICATION, in(Defaults.SSL_CLIENT_AUTHENTICATION_VALID_VALUES:_*), MEDIUM, SslClientAuthDoc) .define(SslCipherSuitesProp, LIST, Collections.emptyList(), MEDIUM, SslCipherSuitesDoc) - .define(SslPrincipalMappingRulesProp, STRING, Defaults.SslPrincipalMappingRules, LOW, SslPrincipalMappingRulesDoc) + .define(SslPrincipalMappingRulesProp, STRING, Defaults.SSL_PRINCIPAL_MAPPING_RULES, LOW, SslPrincipalMappingRulesDoc) .define(SslEngineFactoryClassProp, CLASS, null, LOW, SslEngineFactoryClassDoc) .define(SslAllowDnChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE, LOW, SslAllowDnChangesDoc) .define(SslAllowSanChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, SslAllowSanChangesDoc) /** ********* Sasl Configuration ****************/ - .define(SaslMechanismInterBrokerProtocolProp, STRING, Defaults.SaslMechanismInterBrokerProtocol, MEDIUM, SaslMechanismInterBrokerProtocolDoc) + .define(SaslMechanismInterBrokerProtocolProp, STRING, Defaults.SASL_MECHANISM_INTER_BROKER_PROTOCOL, MEDIUM, SaslMechanismInterBrokerProtocolDoc) .define(SaslJaasConfigProp, PASSWORD, null, MEDIUM, SaslJaasConfigDoc) - .define(SaslEnabledMechanismsProp, LIST, Defaults.SaslEnabledMechanisms, MEDIUM, SaslEnabledMechanismsDoc) + .define(SaslEnabledMechanismsProp, LIST, Defaults.SASL_ENABLED_MECHANISMS, MEDIUM, SaslEnabledMechanismsDoc) .define(SaslServerCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslServerCallbackHandlerClassDoc) .define(SaslClientCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslClientCallbackHandlerClassDoc) .define(SaslLoginClassProp, CLASS, null, MEDIUM, SaslLoginClassDoc) .define(SaslLoginCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslLoginCallbackHandlerClassDoc) .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, SaslKerberosServiceNameDoc) - .define(SaslKerberosKinitCmdProp, STRING, Defaults.SaslKerberosKinitCmd, MEDIUM, SaslKerberosKinitCmdDoc) - .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SaslKerberosTicketRenewWindowFactor, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc) - .define(SaslKerberosTicketRenewJitterProp, DOUBLE, Defaults.SaslKerberosTicketRenewJitter, MEDIUM, SaslKerberosTicketRenewJitterDoc) - .define(SaslKerberosMinTimeBeforeReloginProp, LONG, Defaults.SaslKerberosMinTimeBeforeRelogin, MEDIUM, SaslKerberosMinTimeBeforeReloginDoc) - .define(SaslKerberosPrincipalToLocalRulesProp, LIST, Defaults.SaslKerberosPrincipalToLocalRules, MEDIUM, SaslKerberosPrincipalToLocalRulesDoc) - .define(SaslLoginRefreshWindowFactorProp, DOUBLE, Defaults.SaslLoginRefreshWindowFactor, MEDIUM, SaslLoginRefreshWindowFactorDoc) - .define(SaslLoginRefreshWindowJitterProp, DOUBLE, Defaults.SaslLoginRefreshWindowJitter, MEDIUM, SaslLoginRefreshWindowJitterDoc) - .define(SaslLoginRefreshMinPeriodSecondsProp, SHORT, Defaults.SaslLoginRefreshMinPeriodSeconds, MEDIUM, SaslLoginRefreshMinPeriodSecondsDoc) - .define(SaslLoginRefreshBufferSecondsProp, SHORT, Defaults.SaslLoginRefreshBufferSeconds, MEDIUM, SaslLoginRefreshBufferSecondsDoc) + .define(SaslKerberosKinitCmdProp, STRING, Defaults.SASL_KERBEROS_KINIT_CMD, MEDIUM, SaslKerberosKinitCmdDoc) + .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc) + .define(SaslKerberosTicketRenewJitterProp, DOUBLE, Defaults.SASL_KERBEROS_TICKET_RENEW_JITTER, MEDIUM, SaslKerberosTicketRenewJitterDoc) + .define(SaslKerberosMinTimeBeforeReloginProp, LONG, Defaults.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, MEDIUM, SaslKerberosMinTimeBeforeReloginDoc) + .define(SaslKerberosPrincipalToLocalRulesProp, LIST, Defaults.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES, MEDIUM, SaslKerberosPrincipalToLocalRulesDoc) + .define(SaslLoginRefreshWindowFactorProp, DOUBLE, Defaults.SASL_LOGIN_REFRESH_WINDOW_FACTOR, MEDIUM, SaslLoginRefreshWindowFactorDoc) + .define(SaslLoginRefreshWindowJitterProp, DOUBLE, Defaults.SASL_LOGIN_REFRESH_WINDOW_JITTER, MEDIUM, SaslLoginRefreshWindowJitterDoc) + .define(SaslLoginRefreshMinPeriodSecondsProp, SHORT, Defaults.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, MEDIUM, SaslLoginRefreshMinPeriodSecondsDoc) + .define(SaslLoginRefreshBufferSecondsProp, SHORT, Defaults.SASL_LOGIN_REFRESH_BUFFER_SECONDS, MEDIUM, SaslLoginRefreshBufferSecondsDoc) .define(SaslLoginConnectTimeoutMsProp, INT, null, LOW, SaslLoginConnectTimeoutMsDoc) .define(SaslLoginReadTimeoutMsProp, INT, null, LOW, SaslLoginReadTimeoutMsDoc) - .define(SaslLoginRetryBackoffMaxMsProp, LONG, Defaults.SaslLoginRetryBackoffMaxMs, LOW, SaslLoginRetryBackoffMaxMsDoc) - .define(SaslLoginRetryBackoffMsProp, LONG, Defaults.SaslLoginRetryBackoffMs, LOW, SaslLoginRetryBackoffMsDoc) - .define(SaslOAuthBearerScopeClaimNameProp, STRING, Defaults.SaslOAuthBearerScopeClaimName, LOW, SaslOAuthBearerScopeClaimNameDoc) - .define(SaslOAuthBearerSubClaimNameProp, STRING, Defaults.SaslOAuthBearerSubClaimName, LOW, SaslOAuthBearerSubClaimNameDoc) + .define(SaslLoginRetryBackoffMaxMsProp, LONG, Defaults.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LOW, SaslLoginRetryBackoffMaxMsDoc) + .define(SaslLoginRetryBackoffMsProp, LONG, Defaults.SASL_LOGIN_RETRY_BACKOFF_MS, LOW, SaslLoginRetryBackoffMsDoc) + .define(SaslOAuthBearerScopeClaimNameProp, STRING, Defaults.SASL_OAUTH_BEARER_SCOPE_CLAIM_NAME, LOW, SaslOAuthBearerScopeClaimNameDoc) + .define(SaslOAuthBearerSubClaimNameProp, STRING, Defaults.SASL_OAUTH_BEARER_SUB_CLAIM_NAME, LOW, SaslOAuthBearerSubClaimNameDoc) .define(SaslOAuthBearerTokenEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerTokenEndpointUrlDoc) .define(SaslOAuthBearerJwksEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerJwksEndpointUrlDoc) - .define(SaslOAuthBearerJwksEndpointRefreshMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRefreshMs, LOW, SaslOAuthBearerJwksEndpointRefreshMsDoc) - .define(SaslOAuthBearerJwksEndpointRetryBackoffMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRetryBackoffMs, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMsDoc) - .define(SaslOAuthBearerJwksEndpointRetryBackoffMaxMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRetryBackoffMaxMs, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMaxMsDoc) - .define(SaslOAuthBearerClockSkewSecondsProp, INT, Defaults.SaslOAuthBearerClockSkewSeconds, LOW, SaslOAuthBearerClockSkewSecondsDoc) + .define(SaslOAuthBearerJwksEndpointRefreshMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_REFRESH_MS, LOW, SaslOAuthBearerJwksEndpointRefreshMsDoc) + .define(SaslOAuthBearerJwksEndpointRetryBackoffMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMsDoc) + .define(SaslOAuthBearerJwksEndpointRetryBackoffMaxMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMaxMsDoc) + .define(SaslOAuthBearerClockSkewSecondsProp, INT, Defaults.SASL_OAUTH_BEARER_CLOCK_SKEW_SECONDS, LOW, SaslOAuthBearerClockSkewSecondsDoc) .define(SaslOAuthBearerExpectedAudienceProp, LIST, null, LOW, SaslOAuthBearerExpectedAudienceDoc) .define(SaslOAuthBearerExpectedIssuerProp, STRING, null, LOW, SaslOAuthBearerExpectedIssuerDoc) /** ********* Delegation Token Configuration ****************/ .define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc) .define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc) - .define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DelegationTokenMaxLifeTimeMsDefault, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc) - .define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DelegationTokenExpiryTimeMsDefault, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc) - .define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DelegationTokenExpiryCheckIntervalMsDefault, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc) + .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) /** ********* Password encryption configuration for dynamic configs *********/ .define(PasswordEncoderSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderSecretDoc) .define(PasswordEncoderOldSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderOldSecretDoc) .define(PasswordEncoderKeyFactoryAlgorithmProp, STRING, null, LOW, PasswordEncoderKeyFactoryAlgorithmDoc) - .define(PasswordEncoderCipherAlgorithmProp, STRING, Defaults.PasswordEncoderCipherAlgorithm, LOW, PasswordEncoderCipherAlgorithmDoc) - .define(PasswordEncoderKeyLengthProp, INT, Defaults.PasswordEncoderKeyLength, atLeast(8), LOW, PasswordEncoderKeyLengthDoc) - .define(PasswordEncoderIterationsProp, INT, Defaults.PasswordEncoderIterations, atLeast(1024), LOW, PasswordEncoderIterationsDoc) + .define(PasswordEncoderCipherAlgorithmProp, STRING, Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, LOW, PasswordEncoderCipherAlgorithmDoc) + .define(PasswordEncoderKeyLengthProp, INT, Defaults.PASSWORD_ENCODER_KEY_LENGTH, atLeast(8), LOW, PasswordEncoderKeyLengthDoc) + .define(PasswordEncoderIterationsProp, INT, Defaults.PASSWORD_ENCODER_ITERATIONS, atLeast(1024), LOW, PasswordEncoderIterationsDoc) /** ********* Raft Quorum Configuration *********/ - .define(RaftConfig.QUORUM_VOTERS_CONFIG, LIST, Defaults.QuorumVoters, new RaftConfig.ControllerQuorumVotersValidator(), HIGH, RaftConfig.QUORUM_VOTERS_DOC) - .define(RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, INT, Defaults.QuorumElectionTimeoutMs, null, HIGH, RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_DOC) - .define(RaftConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, INT, Defaults.QuorumFetchTimeoutMs, null, HIGH, RaftConfig.QUORUM_FETCH_TIMEOUT_MS_DOC) - .define(RaftConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, Defaults.QuorumElectionBackoffMs, null, HIGH, RaftConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_DOC) - .define(RaftConfig.QUORUM_LINGER_MS_CONFIG, INT, Defaults.QuorumLingerMs, null, MEDIUM, RaftConfig.QUORUM_LINGER_MS_DOC) - .define(RaftConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, INT, Defaults.QuorumRequestTimeoutMs, null, MEDIUM, RaftConfig.QUORUM_REQUEST_TIMEOUT_MS_DOC) - .define(RaftConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, Defaults.QuorumRetryBackoffMs, null, LOW, RaftConfig.QUORUM_RETRY_BACKOFF_MS_DOC) + .define(RaftConfig.QUORUM_VOTERS_CONFIG, LIST, Defaults.QUORUM_VOTERS, new RaftConfig.ControllerQuorumVotersValidator(), HIGH, RaftConfig.QUORUM_VOTERS_DOC) + .define(RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG, INT, Defaults.QUORUM_ELECTION_TIMEOUT_MS, null, HIGH, RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_DOC) + .define(RaftConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG, INT, Defaults.QUORUM_FETCH_TIMEOUT_MS, null, HIGH, RaftConfig.QUORUM_FETCH_TIMEOUT_MS_DOC) + .define(RaftConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG, INT, Defaults.QUORUM_ELECTION_BACKOFF_MS, null, HIGH, RaftConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_DOC) + .define(RaftConfig.QUORUM_LINGER_MS_CONFIG, INT, Defaults.QUORUM_LINGER_MS, null, MEDIUM, RaftConfig.QUORUM_LINGER_MS_DOC) + .define(RaftConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG, INT, Defaults.QUORUM_REQUEST_TIMEOUT_MS, null, MEDIUM, RaftConfig.QUORUM_REQUEST_TIMEOUT_MS_DOC) + .define(RaftConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG, INT, Defaults.QUORUM_RETRY_BACKOFF_MS, null, LOW, RaftConfig.QUORUM_RETRY_BACKOFF_MS_DOC) /** Internal Configurations **/ // This indicates whether unreleased APIs should be advertised by this node. diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 5b2dffb1a56..c291859d559 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -28,7 +28,7 @@ import java.{time, util} import kafka.integration.KafkaServerTestHarness import kafka.security.authorizer.AclEntry import kafka.server.metadata.KRaftMetadataCache -import kafka.server.{Defaults, DynamicConfig, KafkaConfig} +import kafka.server.{DynamicConfig, KafkaConfig} import kafka.utils.TestUtils._ import kafka.utils.{Log4jController, TestInfoUtils, TestUtils} import org.apache.kafka.clients.HostResolver @@ -44,6 +44,7 @@ import org.apache.kafka.common.resource.{PatternType, ResourcePattern, ResourceT import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT +import org.apache.kafka.server.config.Defaults import org.apache.kafka.storage.internals.log.LogConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} @@ -1352,7 +1353,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) // Increase timeouts to avoid having a rebalance during the test newConsumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.MAX_VALUE.toString) - newConsumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GroupMaxSessionTimeoutMs.toString) + newConsumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GROUP_MAX_SESSION_TIMEOUT_MS.toString) val consumer = createConsumer(configOverrides = newConsumerConfig) try { diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index 0db856804cf..177a89c8c67 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -17,11 +17,12 @@ */ package kafka.network -import kafka.server.{BaseRequestTest, Defaults, KafkaConfig} +import kafka.server.{BaseRequestTest, KafkaConfig} import kafka.utils.{TestInfoUtils, 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.server.config.Defaults import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.params.ParameterizedTest @@ -42,7 +43,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { properties.put(KafkaConfig.ListenersProp, s"$internal://localhost:0, $external://localhost:0") properties.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$internal:PLAINTEXT, $external:PLAINTEXT") properties.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", "2") - properties.put(KafkaConfig.NumNetworkThreadsProp, Defaults.NumNetworkThreads.toString) + properties.put(KafkaConfig.NumNetworkThreadsProp, Defaults.NUM_NETWORK_THREADS.toString) } @BeforeEach @@ -51,7 +52,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.NumNetworkThreads, getNumNetworkThreads(external)) + assertEquals(Defaults.NUM_NETWORK_THREADS, getNumNetworkThreads(external)) } @AfterEach override def tearDown(): Unit = { @@ -69,7 +70,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { @ValueSource(strings = Array("zk", "kraft")) def testDynamicNumNetworkThreads(quorum: String): Unit = { // Increase the base network thread count - val newBaseNetworkThreadsCount = Defaults.NumNetworkThreads + 1 + val newBaseNetworkThreadsCount = Defaults.NUM_NETWORK_THREADS + 1 var props = new Properties props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString) reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)) diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index c67c543651f..ae86fb92153 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.FileRecords import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} @@ -51,8 +52,8 @@ object StressTestLog { scheduler = time.scheduler, time = time, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, brokerTopicStats = new BrokerTopicStats, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 636bfd61b1f..061c907d04c 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -29,6 +29,7 @@ import kafka.utils._ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.{KafkaScheduler, Scheduler} import org.apache.kafka.server.util.CommandLineUtils import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} @@ -220,8 +221,8 @@ object TestLinearWriteSpeed { brokerTopicStats = new BrokerTopicStats, time = Time.SYSTEM, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/admin/DeleteOffsetsConsumerGroupCommandIntegrationTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteOffsetsConsumerGroupCommandIntegrationTest.scala index 125d8b1a656..e7e73a260e1 100644 --- a/core/src/test/scala/unit/kafka/admin/DeleteOffsetsConsumerGroupCommandIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/admin/DeleteOffsetsConsumerGroupCommandIntegrationTest.scala @@ -18,8 +18,6 @@ package kafka.admin import java.util.Properties - -import kafka.server.Defaults import kafka.utils.TestUtils import kafka.utils.TestInfoUtils import org.apache.kafka.clients.consumer.Consumer @@ -33,6 +31,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest @@ -202,7 +201,7 @@ class DeleteOffsetsConsumerGroupCommandIntegrationTest extends ConsumerGroupComm config.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName) // Increase timeouts to avoid having a rebalance during the test config.putIfAbsent(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.MAX_VALUE.toString) - config.putIfAbsent(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GroupMaxSessionTimeoutMs.toString) + config.putIfAbsent(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, Defaults.GROUP_MAX_SESSION_TIMEOUT_MS.toString) new KafkaConsumer(config) } diff --git a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala index 8e53111ebed..774fc768af5 100644 --- a/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/AbstractPartitionTest.scala @@ -17,7 +17,7 @@ package kafka.cluster import kafka.log.LogManager -import kafka.server.{Defaults, MetadataCache} +import kafka.server.MetadataCache import kafka.server.checkpoints.OffsetCheckpoints import kafka.server.metadata.MockConfigRepository import kafka.utils.TestUtils.{MockAlterPartitionListener, MockAlterPartitionManager} @@ -34,6 +34,7 @@ import org.mockito.Mockito.{mock, when} import java.io.File import java.util.Properties import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} @@ -80,7 +81,7 @@ class AbstractPartitionTest { alterPartitionManager = TestUtils.createAlterIsrManager() alterPartitionListener = TestUtils.createIsrChangeListener() partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = interBrokerProtocolVersion, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala index 8523df82023..6f0d8ab110f 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala @@ -35,6 +35,7 @@ import org.apache.kafka.common.requests.FetchRequest import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.server.common.MetadataVersion +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{AppendOrigin, CleanerConfig, FetchIsolation, FetchParams, LogAppendInfo, LogConfig, LogDirFailureChannel, LogSegments, ProducerStateManager, ProducerStateManagerConfig, VerificationGuard} @@ -270,7 +271,7 @@ class PartitionLockTest extends Logging { logManager.startup(Set.empty) val partition = new Partition(topicPartition, - replicaLagTimeMaxMs = kafka.server.Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => 1L, @@ -299,7 +300,7 @@ class PartitionLockTest extends Logging { val segments = new LogSegments(log.topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") val maxTransactionTimeout = 5 * 60 * 1000 - val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) + val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false) val producerStateManager = new ProducerStateManager( log.topicPartition, log.dir, diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index 9a302ea6ea8..7a03fa6970c 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -33,6 +33,7 @@ import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{AlterPartitionResponse, FetchRequest, ListOffsetsRequest, RequestHeader} import org.apache.kafka.common.utils.SystemTime import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.metadata.LeaderRecoveryState import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -416,7 +417,7 @@ class PartitionTest extends AbstractPartitionTest { partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -433,7 +434,7 @@ class PartitionTest extends AbstractPartitionTest { val segments = new LogSegments(log.topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") val maxTransactionTimeoutMs = 5 * 60 * 1000 - val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, true) + val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, true) val producerStateManager = new ProducerStateManager( log.topicPartition, log.dir, @@ -1246,7 +1247,7 @@ class PartitionTest extends AbstractPartitionTest { def testIsUnderMinIsr(): Unit = { configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = interBrokerProtocolVersion, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1337,7 +1338,7 @@ class PartitionTest extends AbstractPartitionTest { def testIsReplicaIsrEligibleWithEmptyReplicaMap(): Unit = { val mockMetadataCache: KRaftMetadataCache = mock(classOf[KRaftMetadataCache]) val partition = spy(new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = interBrokerProtocolVersion, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1569,7 +1570,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1686,7 +1687,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1793,7 +1794,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1885,7 +1886,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -1951,7 +1952,7 @@ class PartitionTest extends AbstractPartitionTest { addBrokerEpochToMockMetadataCache(metadataCache, replicas) val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2107,7 +2108,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2190,7 +2191,7 @@ class PartitionTest extends AbstractPartitionTest { val partition = new Partition( topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.IBP_3_7_IV2, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2568,7 +2569,7 @@ class PartitionTest extends AbstractPartitionTest { ) partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = interBrokerProtocolVersion, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2688,7 +2689,7 @@ class PartitionTest extends AbstractPartitionTest { zkIsrManager.start() val partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = IBP_2_6_IV0, // shouldn't matter, but set this to a ZK isr version localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2780,7 +2781,7 @@ class PartitionTest extends AbstractPartitionTest { // Create new Partition object for same topicPartition val partition2 = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2825,7 +2826,7 @@ class PartitionTest extends AbstractPartitionTest { // Create new Partition object for same topicPartition val partition2 = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -2983,7 +2984,7 @@ class PartitionTest extends AbstractPartitionTest { cleanerConfig = new CleanerConfig(false), time = time) val spyLogManager = spy(logManager) val partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -3022,7 +3023,7 @@ class PartitionTest extends AbstractPartitionTest { }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) val partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), @@ -3064,7 +3065,7 @@ class PartitionTest extends AbstractPartitionTest { }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) val partition = new Partition(topicPartition, - replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, + replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS, interBrokerProtocolVersion = MetadataVersion.latestTesting, localBrokerId = brokerId, () => defaultBrokerEpoch(brokerId), diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala index a5e2d57d87f..1efccc847c9 100644 --- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorTest.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult} import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} +import org.apache.kafka.coordinator.transaction.TransactionStateManagerConfig import org.apache.kafka.server.util.MockScheduler import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -1003,7 +1004,7 @@ class TransactionCoordinatorTest { .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) val expectedTransition = TxnTransitMetadata(producerId, producerId, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, - txnTimeoutMs, PrepareAbort, partitions.toSet, now, now + TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) + txnTimeoutMs, PrepareAbort, partitions.toSet, now, now + TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS) when(transactionManager.appendTransactionToLog(ArgumentMatchers.eq(transactionalId), ArgumentMatchers.eq(coordinatorEpoch), @@ -1014,7 +1015,7 @@ class TransactionCoordinatorTest { ).thenAnswer(_ => {}) coordinator.startup(() => transactionStatePartitionCount, false) - time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) + time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS) scheduler.tick() verify(transactionManager).timedOutTransactions() verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId)) @@ -1063,7 +1064,7 @@ class TransactionCoordinatorTest { .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, metadata)))) coordinator.startup(() => transactionStatePartitionCount, false) - time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) + time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS) scheduler.tick() verify(transactionManager).timedOutTransactions() verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) @@ -1087,7 +1088,7 @@ class TransactionCoordinatorTest { val bumpedEpoch = (producerEpoch + 1).toShort val expectedTransition = TxnTransitMetadata(producerId, producerId, bumpedEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, - PrepareAbort, partitions.toSet, now, now + TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) + PrepareAbort, partitions.toSet, now, now + TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS) when(transactionManager.appendTransactionToLog(ArgumentMatchers.eq(transactionalId), ArgumentMatchers.eq(coordinatorEpoch), @@ -1098,7 +1099,7 @@ class TransactionCoordinatorTest { ).thenAnswer(_ => capturedErrorsCallback.getValue.apply(Errors.NOT_ENOUGH_REPLICAS)) coordinator.startup(() => transactionStatePartitionCount, false) - time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) + time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS) scheduler.tick() verify(transactionManager).timedOutTransactions() diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala index 5cb66210376..95c7237d79f 100644 --- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.junit.jupiter.api.{AfterEach, Tag} @@ -112,8 +113,8 @@ abstract class AbstractLogCleanerIntegrationTest { time = time, brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index 3d2cb3068e1..a39f33cf9ee 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -22,6 +22,7 @@ import kafka.utils._ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} @@ -63,8 +64,8 @@ class BrokerCompressionTest { time = time, brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala index 9656ae67a6e..0f74694245e 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala @@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, LogDirFailureChannel, LogSegment, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig} import org.junit.jupiter.api.Assertions._ @@ -53,7 +54,7 @@ class LogCleanerManagerTest extends Logging { val logConfig: LogConfig = new LogConfig(logProps) val time = new MockTime(1400000000000L, 1000L) // Tue May 13 16:53:20 UTC 2014 for `currentTimeMs` val offset = 999 - val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) + val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false) val cleanerCheckpoints: mutable.Map[TopicPartition, Long] = mutable.Map[TopicPartition, Long]() @@ -104,7 +105,7 @@ class LogCleanerManagerTest extends Logging { val logDirFailureChannel = new LogDirFailureChannel(10) val config = createLowRetentionLogConfig(logSegmentSize, TopicConfig.CLEANUP_POLICY_COMPACT) val maxTransactionTimeoutMs = 5 * 60 * 1000 - val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS val segments = new LogSegments(tp) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) @@ -813,7 +814,7 @@ class LogCleanerManagerTest extends Logging { brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, producerStateManagerConfig = producerStateManagerConfig, - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true) @@ -867,7 +868,7 @@ class LogCleanerManagerTest extends Logging { brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, producerStateManagerConfig = producerStateManagerConfig, - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala index 48dfd8c75cf..bcf9fc02242 100644 --- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.Utils +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.metrics.KafkaMetricsGroup import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, CleanerConfig, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetMap, ProducerStateManager, ProducerStateManagerConfig} @@ -60,7 +61,7 @@ class LogCleanerTest extends Logging { val throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time) val tombstoneRetentionMs = 86400000 val largeTimestamp = Long.MaxValue - tombstoneRetentionMs - 1 - val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) + val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false) @AfterEach def teardown(): Unit = { @@ -163,7 +164,7 @@ class LogCleanerTest extends Logging { val topicPartition = UnifiedLog.parseTopicPartitionName(dir) val logDirFailureChannel = new LogDirFailureChannel(10) val maxTransactionTimeoutMs = 5 * 60 * 1000 - val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS val logSegments = new LogSegments(topicPartition) val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.recordVersion, "") val producerStateManager = new ProducerStateManager(topicPartition, dir, @@ -2031,7 +2032,7 @@ class LogCleanerTest extends Logging { brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, producerStateManagerConfig = producerStateManagerConfig, - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala index 7935cc761dc..0edf5935bb2 100644 --- a/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConcurrencyTest.scala @@ -24,6 +24,7 @@ import kafka.utils.TestUtils import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.utils.{Time, Utils} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.junit.jupiter.api.Assertions._ @@ -151,8 +152,8 @@ class LogConcurrencyTest { brokerTopicStats = brokerTopicStats, time = Time.SYSTEM, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala index 5232f3b10b5..f44285bd908 100644 --- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.record.{CompressionType, ControlRecordType, Defau import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0 +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.{MockTime, Scheduler} import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{AbortedTxn, CleanerConfig, EpochEntry, FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments, LogStartOffsetIncrementReason, OffsetIndex, ProducerStateManager, ProducerStateManagerConfig, SnapshotFile} @@ -55,8 +56,8 @@ class LogLoaderTest { var config: KafkaConfig = _ val brokerTopicStats = new BrokerTopicStats val maxTransactionTimeoutMs: Int = 5 * 60 * 1000 - val producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) - val producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false) + val producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS val tmpDir = TestUtils.tempDir() val logDir = TestUtils.randomPartitionLogDir(tmpDir) var logsToClose: Seq[UnifiedLog] = Seq() @@ -98,7 +99,7 @@ class LogLoaderTest { val logDirFailureChannel = new LogDirFailureChannel(logDirs.size) val maxTransactionTimeoutMs = 5 * 60 * 1000 - val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS // Create a LogManager with some overridden methods to facilitate interception of clean shutdown // flag and to inject an error @@ -345,7 +346,7 @@ class LogLoaderTest { def createLogWithInterceptedReads(recoveryPoint: Long): UnifiedLog = { val maxTransactionTimeoutMs = 5 * 60 * 1000 - val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logDirFailureChannel = new LogDirFailureChannel(10) // Intercept all segment read calls @@ -506,7 +507,7 @@ class LogLoaderTest { firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) - val maxProducerIdExpirationMs = kafka.server.Defaults.ProducerIdExpirationMs + val maxProducerIdExpirationMs = Defaults.PRODUCER_ID_EXPIRATION_MS mockTime.sleep(maxProducerIdExpirationMs) assertEquals(Optional.empty(), log.producerStateManager.lastEntry(producerId)) diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index c93f9692af2..c0be447a7d7 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.OffsetOutOfRangeException import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} +import org.apache.kafka.server.config.Defaults import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.mockito.ArgumentMatchers.any @@ -793,7 +794,7 @@ class LogManagerTest { val segmentBytes = 1024 val log = LogTestUtils.createLog(tpFile, logConfig, brokerTopicStats, time.scheduler, time, 0, 0, - 5 * 60 * 1000, new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs) + 5 * 60 * 1000, new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS) assertTrue(expectedSegmentsPerLog > 0) // calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024 @@ -929,7 +930,7 @@ class LogManagerTest { recoveryPoint = 0, maxTransactionTimeoutMs = 5 * 60 * 1000, producerStateManagerConfig = new ProducerStateManagerConfig(5 * 60 * 1000, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, scheduler = mockTime.scheduler, time = mockTime, brokerTopicStats = mockBrokerTopicStats, diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala index 11fff517b43..ebdcab0ccf9 100644 --- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -22,6 +22,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Time, Utils} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpoint import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache import org.apache.kafka.storage.internals.log.{BatchMetadata, EpochEntry, LogConfig, LogFileUtils, LogSegment, LogSegmentOffsetOverflowException, ProducerStateEntry, ProducerStateManager, ProducerStateManagerConfig, RollParams} @@ -607,7 +608,7 @@ class LogSegmentTest { topicPartition, logDir, 5 * 60 * 1000, - new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), + new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), new MockTime() ) } diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala index c3f630e7646..4b0f3f59869 100644 --- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala +++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala @@ -31,6 +31,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} import java.nio.file.Files import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import org.apache.kafka.common.config.TopicConfig +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.Scheduler import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile import org.apache.kafka.storage.internals.log.{AbortedTxn, AppendOrigin, FetchDataInfo, FetchIsolation, LazyIndex, LogAppendInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetsListener, LogSegment, ProducerStateManager, ProducerStateManagerConfig, TransactionIndex} @@ -91,8 +92,8 @@ object LogTestUtils { logStartOffset: Long = 0L, recoveryPoint: Long = 0L, maxTransactionTimeoutMs: Int = 5 * 60 * 1000, - producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, lastShutdownClean: Boolean = true, topicId: Option[Uuid] = None, keepPartitionMetadataFile: Boolean = true, diff --git a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala index f6e29a73428..03aa847ded7 100644 --- a/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/ProducerStateManagerTest.scala @@ -29,6 +29,7 @@ import org.apache.kafka.common.errors._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record._ import org.apache.kafka.common.utils.{MockTime, Utils} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.storage.internals.log.{AppendOrigin, CompletedTxn, LogFileUtils, LogOffsetMetadata, ProducerAppendInfo, ProducerStateEntry, ProducerStateManager, ProducerStateManagerConfig, TxnMetadata, VerificationStateEntry} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} @@ -46,7 +47,7 @@ class ProducerStateManagerTest { private val partition = new TopicPartition("test", 0) private val producerId = 1L private val maxTransactionTimeoutMs = 5 * 60 * 1000 - private val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, true) + private val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, true) private val lateTransactionTimeoutMs = maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS private val time = new MockTime diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala index 0b9533380b1..5a476574e18 100755 --- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala +++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter import org.apache.kafka.common.record._ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler} @@ -62,7 +63,7 @@ class UnifiedLogTest { val logDir = TestUtils.randomPartitionLogDir(tmpDir) val mockTime = new MockTime() var logsToClose: Seq[UnifiedLog] = Seq() - val producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) + val producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false) def metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala @BeforeEach @@ -4142,7 +4143,7 @@ class UnifiedLogTest { time: Time = mockTime, maxTransactionTimeoutMs: Int = 60 * 60 * 1000, producerStateManagerConfig: ProducerStateManagerConfig = producerStateManagerConfig, - producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, lastShutdownClean: Boolean = true, topicId: Option[Uuid] = None, keepPartitionMetadataFile: Boolean = true, 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 306f1bae38a..f829ce3570b 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 @@ -23,7 +23,7 @@ import java.util.{Base64, Properties} import kafka.network.RequestChannel.Session import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils} import kafka.security.authorizer.AclEntry.WildcardHost -import kafka.server.{CreateTokenResult, Defaults, DelegationTokenManager, DelegationTokenManagerZk, KafkaConfig, QuorumTestHarness} +import kafka.server.{CreateTokenResult, DelegationTokenManager, DelegationTokenManagerZk, KafkaConfig, QuorumTestHarness} import kafka.utils.TestUtils import kafka.zk.KafkaZkClient import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclOperation} @@ -39,6 +39,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok import org.apache.kafka.common.security.token.delegation.{DelegationToken, TokenInformation} import org.apache.kafka.common.utils.{MockTime, SecurityUtils, Time} import org.apache.kafka.server.authorizer._ +import org.apache.kafka.server.config.Defaults import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} @@ -57,8 +58,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness { val tokenManagers = Buffer[DelegationTokenManager]() val secretKey = "secretKey" - val maxLifeTimeMsDefault = Defaults.DelegationTokenMaxLifeTimeMsDefault - val renewTimeMsDefault = Defaults.DelegationTokenExpiryTimeMsDefault + val maxLifeTimeMsDefault = Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS + val renewTimeMsDefault = Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS var tokenCache: DelegationTokenCache = _ var props: Properties = _ diff --git a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala index e6537edb025..be065f5b8ca 100755 --- a/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicBrokerConfigTest.scala @@ -34,6 +34,7 @@ import org.apache.kafka.common.config.{ConfigException, SslConfigs} import org.apache.kafka.common.metrics.{JmxReporter, Metrics} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.server.authorizer._ +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.KafkaScheduler @@ -630,7 +631,7 @@ class DynamicBrokerConfigTest { val config = KafkaConfig(props) config.dynamicConfig.initialize(None, None) - assertEquals(Defaults.MaxConnections, config.maxConnections) + assertEquals(Defaults.MAX_CONNECTIONS, config.maxConnections) assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, config.messageMaxBytes) var newProps = new Properties() @@ -647,7 +648,7 @@ class DynamicBrokerConfigTest { config.dynamicConfig.updateDefaultConfig(newProps) // Invalid value should be skipped and reassigned as default value - assertEquals(Defaults.MaxConnections, config.maxConnections) + assertEquals(Defaults.MAX_CONNECTIONS, config.maxConnections) // Even if One property is invalid, the below should get correctly updated. assertEquals(1111, config.messageMaxBytes) } @@ -842,8 +843,8 @@ class TestDynamicThreadPool() extends BrokerReconfigurable { } override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { - assertEquals(Defaults.NumIoThreads, oldConfig.numIoThreads) - assertEquals(Defaults.BackgroundThreads, oldConfig.backgroundThreads) + assertEquals(Defaults.NUM_IO_THREADS, oldConfig.numIoThreads) + assertEquals(Defaults.BACKGROUND_THREADS, oldConfig.backgroundThreads) assertEquals(10, newConfig.numIoThreads) assertEquals(100, newConfig.backgroundThreads) diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala index c9ba76c9e35..e769c7ddaed 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala @@ -99,7 +99,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.common.{Features, MetadataVersion} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1} -import org.apache.kafka.server.config.ConfigType +import org.apache.kafka.server.config.{ConfigType, Defaults} import org.apache.kafka.server.metrics.ClientMetricsTestUtils import org.apache.kafka.server.util.{FutureUtils, MockTime} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchParams, FetchPartitionData, LogConfig} @@ -3118,7 +3118,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) + ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) when(groupCoordinator.completeTransaction( @@ -3127,7 +3127,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.ABORT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) + ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) )).thenReturn(CompletableFuture.completedFuture[Void](null)) val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = @@ -3136,7 +3136,7 @@ class KafkaApisTest extends Logging { ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) when(replicaManager.appendRecords( - ArgumentMatchers.eq(Defaults.RequestTimeoutMs.toLong), + ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong), ArgumentMatchers.eq(-1), ArgumentMatchers.eq(true), ArgumentMatchers.eq(AppendOrigin.COORDINATOR), @@ -3237,7 +3237,7 @@ class KafkaApisTest extends Logging { ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(0), ArgumentMatchers.eq(TransactionResult.COMMIT), - ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) + ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS)) )).thenReturn(FutureUtils.failedFuture[Void](error.exception())) kafkaApis = createKafkaApis(overrideProperties = Map( KafkaConfig.NewGroupCoordinatorEnableProp -> "true" diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala index d2c8e663764..11f2752384a 100644 --- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala +++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala @@ -36,6 +36,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} import org.apache.kafka.server.common.ApiMessageAndVersion +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.MockTime import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} @@ -76,8 +77,8 @@ class DumpLogSegmentsTest { time = time, brokerTopicStats = new BrokerTopicStats, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, logDirFailureChannel = new LogDirFailureChannel(10), topicId = None, keepPartitionMetadataFile = true diff --git a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala index cd05c53fc13..1073ad0082b 100755 --- a/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala +++ b/core/src/test/scala/unit/kafka/utils/PasswordEncoderTest.scala @@ -19,10 +19,9 @@ package kafka.utils import javax.crypto.SecretKeyFactory - -import kafka.server.Defaults import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.config.types.Password +import org.apache.kafka.server.config.Defaults import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -32,9 +31,9 @@ class PasswordEncoderTest { def testEncodeDecode(): Unit = { val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), None, - Defaults.PasswordEncoderCipherAlgorithm, - Defaults.PasswordEncoderKeyLength, - Defaults.PasswordEncoderIterations) + Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, + Defaults.PASSWORD_ENCODER_KEY_LENGTH, + Defaults.PASSWORD_ENCODER_ITERATIONS) val password = "test-password" val encoded = encoder.encode(new Password(password)) val encodedMap = CoreUtils.parseCsvMap(encoded) @@ -96,7 +95,7 @@ class PasswordEncoderTest { keyFactoryAlg, cipherAlg, keyLength, - Defaults.PasswordEncoderIterations) + Defaults.PASSWORD_ENCODER_ITERATIONS) val password = "test-password" val encoded = encoder.encode(new Password(password)) verifyEncodedPassword(encoder, password, encoded) @@ -107,10 +106,10 @@ class PasswordEncoderTest { verifyEncodeDecode(keyFactoryAlg = None, "AES/CBC/PKCS5Padding", keyLength = 128) verifyEncodeDecode(keyFactoryAlg = None, "AES/CFB/PKCS5Padding", keyLength = 128) verifyEncodeDecode(keyFactoryAlg = None, "AES/OFB/PKCS5Padding", keyLength = 128) - verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA1"), Defaults.PasswordEncoderCipherAlgorithm, keyLength = 128) + verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA1"), Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, keyLength = 128) verifyEncodeDecode(keyFactoryAlg = None, "AES/GCM/NoPadding", keyLength = 128) - verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA256"), Defaults.PasswordEncoderCipherAlgorithm, keyLength = 128) - verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA512"), Defaults.PasswordEncoderCipherAlgorithm, keyLength = 128) + verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA256"), Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, keyLength = 128) + verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA512"), Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, keyLength = 128) } private def verifyEncodedPassword(encoder: PasswordEncoder, password: String, encoded: String): Unit = { diff --git a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala index eb9c03343e5..0bf8fe34762 100644 --- a/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala +++ b/core/src/test/scala/unit/kafka/utils/SchedulerTest.scala @@ -22,6 +22,7 @@ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import kafka.log.{LocalLog, LogLoader, UnifiedLog} import kafka.server.BrokerTopicStats import kafka.utils.TestUtils.retry +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.util.{KafkaScheduler, MockTime} import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, LogSegments, ProducerStateManager, ProducerStateManagerConfig} import org.junit.jupiter.api.Assertions._ @@ -133,8 +134,8 @@ class SchedulerTest { val logConfig = new LogConfig(new Properties()) val brokerTopicStats = new BrokerTopicStats val maxTransactionTimeoutMs = 5 * 60 * 1000 - val maxProducerIdExpirationMs = kafka.server.Defaults.ProducerIdExpirationMs - val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs + val maxProducerIdExpirationMs = Defaults.PRODUCER_ID_EXPIRATION_MS + val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val logDirFailureChannel = new LogDirFailureChannel(10) val segments = new LogSegments(topicPartition) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 1d7e41bf545..880e4ab9dab 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -74,6 +74,7 @@ import org.apache.kafka.metadata.properties.MetaProperties import org.apache.kafka.server.ControllerRequestCompletionHandler import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} +import org.apache.kafka.server.config.Defaults import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.util.MockTime import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} @@ -1459,8 +1460,8 @@ object TestUtils extends Logging { flushStartOffsetCheckpointMs = 10000L, retentionCheckMs = 1000L, maxTransactionTimeoutMs = 5 * 60 * 1000, - producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, transactionVerificationEnabled), - producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, + producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, transactionVerificationEnabled), + producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS, scheduler = time.scheduler, time = time, brokerTopicStats = new BrokerTopicStats, diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetConfig.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetConfig.java index e15ce64cd82..bb8a43b8f26 100644 --- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetConfig.java +++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/OffsetConfig.java @@ -94,4 +94,4 @@ public class OffsetConfig { DEFAULT_OFFSETS_TOPIC_COMPRESSION_TYPE, DEFAULT_OFFSET_COMMIT_TIMEOUT_MS, DEFAULT_OFFSET_COMMIT_REQUIRED_ACKS); } -} \ No newline at end of file +} 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 new file mode 100644 index 00000000000..f6682ac2f3f --- /dev/null +++ b/server/src/main/java/org/apache/kafka/server/config/Defaults.java @@ -0,0 +1,278 @@ +/* + * 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.coordinator.group.Group; +import org.apache.kafka.coordinator.group.assignor.RangeAssignor; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslClientAuth; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.config.internals.BrokerSecurityConfigs; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.security.auth.KafkaPrincipalBuilder; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder; +import org.apache.kafka.coordinator.group.OffsetConfig; +import org.apache.kafka.coordinator.transaction.TransactionLogConfig; +import org.apache.kafka.coordinator.transaction.TransactionStateManagerConfig; +import org.apache.kafka.raft.RaftConfig; +import org.apache.kafka.server.common.MetadataVersion; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +public class Defaults { + /** ********* Zookeeper Configuration *********/ + public static final int ZK_SESSION_TIMEOUT_MS = 18000; + public static final boolean ZK_ENABLE_SECURE_ACLS = false; + public static final int ZK_MAX_IN_FLIGHT_REQUESTS = 10; + public static final boolean ZK_SSL_CLIENT_ENABLE = false; + public static final String ZK_SSL_PROTOCOL = "TLSv1.2"; + public static final String ZK_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = "HTTPS"; + public static final boolean ZK_SSL_CRL_ENABLE = false; + public static final boolean ZK_SSL_OCSP_ENABLE = false; + + /** ********* 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 int INITIAL_BROKER_REGISTRATION_TIMEOUT_MS = 60000; + public static final int BROKER_HEARTBEAT_INTERVAL_MS = 2000; + public static final int BROKER_SESSION_TIMEOUT_MS = 9000; + public static final int METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES = 20 * 1024 * 1024; + public static final long METADATA_SNAPSHOT_MAX_INTERVAL_MS = TimeUnit.HOURS.toMillis(1); + public static final int METADATA_MAX_IDLE_INTERVAL_MS = 500; + public static final int METADATA_MAX_RETENTION_BYTES = 100 * 1024 * 1024; + public static final boolean DELETE_TOPIC_ENABLE = true; + /** ********* KRaft mode configs *********/ + public static final int EMPTY_NODE_ID = -1; + public static final long SERVER_MAX_STARTUP_TIME_MS = Long.MAX_VALUE; + + /** ********* Authorizer Configuration *********/ + public static final String AUTHORIZER_CLASS_NAME = ""; + + /** ********* Socket Server Configuration *********/ + public static final String LISTENERS = "PLAINTEXT://:9092"; + //TODO: Replace this with EndPoint.DefaultSecurityProtocolMap once EndPoint is out of core. + public static final String LISTENER_SECURITY_PROTOCOL_MAP = Arrays.stream(SecurityProtocol.values()) + .collect(Collectors.toMap(sp -> ListenerName.forSecurityProtocol(sp), sp -> sp)) + .entrySet() + .stream() + .map(entry -> entry.getKey().value() + ":" + entry.getValue().name()) + .collect(Collectors.joining(",")); + public static final int SOCKET_SEND_BUFFER_BYTES = 100 * 1024; + public static final int SOCKET_RECEIVE_BUFFER_BYTES = 100 * 1024; + public static final int SOCKET_REQUEST_MAX_BYTES = 100 * 1024 * 1024; + public static final int SOCKET_LISTEN_BACKLOG_SIZE = 50; + public static final int MAX_CONNECTIONS_PER_IP = Integer.MAX_VALUE; + public static final String MAX_CONNECTIONS_PER_IP_OVERRIDES = ""; + public static final int MAX_CONNECTIONS = Integer.MAX_VALUE; + public static final int MAX_CONNECTION_CREATION_RATE = Integer.MAX_VALUE; + public static final long CONNECTIONS_MAX_IDLE_MS = 10 * 60 * 1000L; + 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; + public static final int FAILED_AUTHENTICATION_DELAY_MS = 100; + + /** ********* Log Configuration *********/ + public static final int NUM_PARTITIONS = 1; + public static final String LOG_DIR = "/tmp/kafka-logs"; + public static final long LOG_CLEANUP_INTERVAL_MS = 5 * 60 * 1000L; + public static final int LOG_CLEANER_THREADS = 1; + public static final double LOG_CLEANER_IO_MAX_BYTES_PER_SECOND = Double.MAX_VALUE; + public static final long LOG_CLEANER_DEDUPE_BUFFER_SIZE = 128 * 1024 * 1024L; + public static final int LOG_CLEANER_IO_BUFFER_SIZE = 512 * 1024; + public static final double LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR = 0.9d; + public static final int LOG_CLEANER_BACKOFF_MS = 15 * 1000; + public static final boolean LOG_CLEANER_ENABLE = true; + public static final int LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS = 60000; + public static final int LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS = 60000; + public static final int NUM_RECOVERY_THREADS_PER_DATA_DIR = 1; + public static final boolean AUTO_CREATE_TOPICS_ENABLE = true; + + /** ********* Replication configuration *********/ + public static final int CONTROLLER_SOCKET_TIMEOUT_MS = REQUEST_TIMEOUT_MS; + public static final int REPLICATION_FACTOR = 1; + public static final long REPLICA_LAG_TIME_MAX_MS = 30000L; + public static final int REPLICA_SOCKET_TIMEOUT_MS = 30 * 1000; + public static final int REPLICA_SOCKET_RECEIVE_BUFFER_BYTES = 64 * 1024; + public static final int REPLICA_FETCH_MAX_BYTES = 1024 * 1024; + public static final int REPLICA_FETCH_WAIT_MAX_MS = 500; + public static final int REPLICA_FETCH_MIN_BYTES = 1; + public static final int REPLICA_FETCH_RESPONSE_MAX_BYTES = 10 * 1024 * 1024; + public static final int NUM_REPLICA_FETCHERS = 1; + public static final int REPLICA_FETCH_BACKOFF_MS = 1000; + public static final long REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS = 5000L; + public static final int FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS = 1000; + public static final int PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS = 1000; + public static final int DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS = 1; + public static final boolean AUTO_LEADER_REBALANCE_ENABLE = true; + public static final int LEADER_IMBALANCE_PER_BROKER_PERCENTAGE = 10; + public static final int LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS = 300; + public static final String INTER_BROKER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT.toString(); + public static final String INTER_BROKER_PROTOCOL_VERSION = MetadataVersion.latestProduction().version(); + + /** ********* 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; + + /** ********* Group coordinator configuration *********/ + public static final int GROUP_MIN_SESSION_TIMEOUT_MS = 6000; + public static final int GROUP_MAX_SESSION_TIMEOUT_MS = 1800000; + public static final int GROUP_INITIAL_REBALANCE_DELAY_MS = 3000; + public static final int GROUP_MAX_SIZE = Integer.MAX_VALUE; + + /** ********* New group coordinator configs *********/ + public static final boolean NEW_GROUP_COORDINATOR_ENABLE = false; + public static final List GROUP_COORDINATOR_REBALANCE_PROTOCOLS = Collections.singletonList(Group.GroupType.CLASSIC.toString()); + public static final int GROUP_COORDINATOR_NUM_THREADS = 1; + + /** ********* Consumer group configs *********/ + public static final int CONSUMER_GROUP_SESSION_TIMEOUT_MS = 45000; + public static final int CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS = 45000; + public static final int CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS = 60000; + public static final int CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS = 5000; + public static final int CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS = 5000; + public static final int CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS = 15000; + public static final int CONSUMER_GROUP_MAX_SIZE = Integer.MAX_VALUE; + public static final List CONSUMER_GROUP_ASSIGNORS = Collections.singletonList(RangeAssignor.class.getName()); + + /** ********* Offset management configuration *********/ + public static final int OFFSET_METADATA_MAX_SIZE = OffsetConfig.DEFAULT_MAX_METADATA_SIZE; + public static final int OFFSETS_LOAD_BUFFER_SIZE = OffsetConfig.DEFAULT_LOAD_BUFFER_SIZE; + public static final short OFFSETS_TOPIC_REPLICATION_FACTOR = OffsetConfig.DEFAULT_OFFSETS_TOPIC_REPLICATION_FACTOR; + public static final int OFFSETS_TOPIC_PARTITIONS = OffsetConfig.DEFAULT_OFFSETS_TOPIC_NUM_PARTITIONS; + public static final int OFFSETS_TOPIC_SEGMENT_BYTES = OffsetConfig.DEFAULT_OFFSETS_TOPIC_SEGMENT_BYTES; + public static final int OFFSETS_TOPIC_COMPRESSION_CODEC = OffsetConfig.DEFAULT_OFFSETS_TOPIC_COMPRESSION_TYPE.id; + public static final int OFFSETS_RETENTION_MINUTES = 7 * 24 * 60; + public static final long OFFSETS_RETENTION_CHECK_INTERVAL_MS = OffsetConfig.DEFAULT_OFFSETS_RETENTION_CHECK_INTERVAL_MS; + public static final int OFFSET_COMMIT_TIMEOUT_MS = OffsetConfig.DEFAULT_OFFSET_COMMIT_TIMEOUT_MS; + public static final short OFFSET_COMMIT_REQUIRED_ACKS = OffsetConfig.DEFAULT_OFFSET_COMMIT_REQUIRED_ACKS; + + /** ********* Transaction management configuration *********/ + public static final int TRANSACTIONAL_ID_EXPIRATION_MS = TransactionStateManagerConfig.DEFAULT_TRANSACTIONAL_ID_EXPIRATION_MS; + public static final int TRANSACTIONS_MAX_TIMEOUT_MS = TransactionStateManagerConfig.DEFAULT_TRANSACTIONS_MAX_TIMEOUT_MS; + public static final int TRANSACTIONS_TOPIC_MIN_ISR = TransactionLogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS; + public static final int TRANSACTIONS_LOAD_BUFFER_SIZE = TransactionLogConfig.DEFAULT_LOAD_BUFFER_SIZE; + public static final short TRANSACTIONS_TOPIC_REPLICATION_FACTOR = TransactionLogConfig.DEFAULT_REPLICATION_FACTOR; + public static final int TRANSACTIONS_TOPIC_PARTITIONS = TransactionLogConfig.DEFAULT_NUM_PARTITIONS; + public static final int TRANSACTIONS_TOPIC_SEGMENT_BYTES = TransactionLogConfig.DEFAULT_SEGMENT_BYTES; + public static final int TRANSACTIONS_ABORT_TIMED_OUT_CLEANUP_INTERVAL_MS = TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS; + public static final int TRANSACTIONS_REMOVE_EXPIRED_CLEANUP_INTERVAL_MS = TransactionStateManagerConfig.DEFAULT_REMOVE_EXPIRED_TRANSACTIONAL_IDS_INTERVAL_MS; + public static final boolean TRANSACTION_PARTITION_VERIFICATION_ENABLE = true; + public static final int PRODUCER_ID_EXPIRATION_MS = 86400000; + public static final int PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS = 600000; + + /** ********* Fetch Configuration *********/ + public static final int MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS = 1000; + public static final int FETCH_MAX_BYTES = 55 * 1024 * 1024; + + /** ********* Quota Configuration *********/ + public static final int NUM_QUOTA_SAMPLES = ClientQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES; + public static final int QUOTA_WINDOW_SIZE_SECONDS = ClientQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS; + public static final int NUM_REPLICATION_QUOTA_SAMPLES = ReplicationQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES; + public static final int REPLICATION_QUOTA_WINDOW_SIZE_SECONDS = ReplicationQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS; + public static final int NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES = ReplicationQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES; + public static final int ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS = ReplicationQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS; + public static final int NUM_CONTROLLER_QUOTA_SAMPLES = ClientQuotaManagerConfig.DEFAULT_NUM_QUOTA_SAMPLES; + public static final int CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS = ClientQuotaManagerConfig.DEFAULT_QUOTA_WINDOW_SIZE_SECONDS; + + /** ********* Kafka Metrics Configuration *********/ + public static final int METRIC_NUM_SAMPLES = 2; + public static final int METRIC_SAMPLE_WINDOW_MS = 30000; + public static final String METRIC_REPORTER_CLASSES = ""; + public static final String METRIC_RECORDING_LEVEL = Sensor.RecordingLevel.INFO.toString(); + public static final boolean AUTO_INCLUDE_JMX_REPORTER = true; + + /** ********* Kafka Yammer Metrics Reporter Configuration *********/ + public static final String KAFKA_METRIC_REPORTER_CLASSES = ""; + public static final int KAFKA_METRICS_POLLING_INTERVAL_SECONDS = 10; + + + /** ********* Kafka Client Telemetry Metrics Configuration *********/ + public static final int CLIENT_TELEMETRY_MAX_BYTES = 1024 * 1024; + + /** ********* SSL configuration *********/ + public static final String SSL_PROTOCOL = SslConfigs.DEFAULT_SSL_PROTOCOL; + public static final String SSL_ENABLED_PROTOCOLS = SslConfigs.DEFAULT_SSL_ENABLED_PROTOCOLS; + public static final String SSL_KEYSTORE_TYPE = SslConfigs.DEFAULT_SSL_KEYSTORE_TYPE; + public static final String SSL_TRUSTSTORE_TYPE = SslConfigs.DEFAULT_SSL_TRUSTSTORE_TYPE; + public static final String SSL_KEY_MANAGER_ALGORITHM = SslConfigs.DEFAULT_SSL_KEYMANGER_ALGORITHM; + public static final String SSL_TRUST_MANAGER_ALGORITHM = SslConfigs.DEFAULT_SSL_TRUSTMANAGER_ALGORITHM; + public static final String SSL_ENDPOINT_IDENTIFICATION_ALGORITHM = SslConfigs.DEFAULT_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM; + public static final String SSL_CLIENT_AUTHENTICATION = SslClientAuth.NONE.name().toLowerCase(Locale.ROOT); + public static final String[] SSL_CLIENT_AUTHENTICATION_VALID_VALUES = SslClientAuth.VALUES.stream() + .map(v -> v.toString().toLowerCase(Locale.ROOT)).toArray(String[]::new); + public static final String SSL_PRINCIPAL_MAPPING_RULES = BrokerSecurityConfigs.DEFAULT_SSL_PRINCIPAL_MAPPING_RULES; + + /** ********* General Security Configuration *********/ + public static final long CONNECTIONS_MAX_REAUTH_MS = 0L; + public static final int SERVER_MAX_RECEIVE_SIZE = BrokerSecurityConfigs.DEFAULT_SASL_SERVER_MAX_RECEIVE_SIZE; + public static final Class PRINCIPAL_BUILDER = DefaultKafkaPrincipalBuilder.class; + + /** ********* Sasl configuration *********/ + public static final String SASL_MECHANISM_INTER_BROKER_PROTOCOL = SaslConfigs.DEFAULT_SASL_MECHANISM; + public static final List SASL_ENABLED_MECHANISMS = BrokerSecurityConfigs.DEFAULT_SASL_ENABLED_MECHANISMS; + public static final String SASL_KERBEROS_KINIT_CMD = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD; + public static final double SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR; + public static final double SASL_KERBEROS_TICKET_RENEW_JITTER = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER; + public static final long SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN = SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN; + public static final List SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES = BrokerSecurityConfigs.DEFAULT_SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES; + public static final double SASL_LOGIN_REFRESH_WINDOW_FACTOR = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_FACTOR; + public static final double SASL_LOGIN_REFRESH_WINDOW_JITTER = SaslConfigs.DEFAULT_LOGIN_REFRESH_WINDOW_JITTER; + public static final short SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS = SaslConfigs.DEFAULT_LOGIN_REFRESH_MIN_PERIOD_SECONDS; + public static final short SASL_LOGIN_REFRESH_BUFFER_SECONDS = SaslConfigs.DEFAULT_LOGIN_REFRESH_BUFFER_SECONDS; + public static final long SASL_LOGIN_RETRY_BACKOFF_MAX_MS = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MAX_MS; + public static final long SASL_LOGIN_RETRY_BACKOFF_MS = SaslConfigs.DEFAULT_SASL_LOGIN_RETRY_BACKOFF_MS; + public static final String SASL_OAUTH_BEARER_SCOPE_CLAIM_NAME = SaslConfigs.SASL_OAUTHBEARER_SCOPE_CLAIM_NAME; + public static final String SASL_OAUTH_BEARER_SUB_CLAIM_NAME = SaslConfigs.SASL_OAUTHBEARER_SUB_CLAIM_NAME; + public static final long SASL_OAUTH_BEARER_JWKS_ENDPOINT_REFRESH_MS = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_REFRESH_MS; + public static final long SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS; + public static final long SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS; + public static final int SASL_OAUTH_BEARER_CLOCK_SKEW_SECONDS = SaslConfigs.DEFAULT_SASL_OAUTHBEARER_CLOCK_SKEW_SECONDS; + + /** ********* 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; + + /** ********* Password Encryption Configuration for Dynamic Configs *********/ + public static final String PASSWORD_ENCODER_CIPHER_ALGORITHM = "AES/CBC/PKCS5Padding"; + public static final int PASSWORD_ENCODER_KEY_LENGTH = 128; + public static final int PASSWORD_ENCODER_ITERATIONS = 4096; + + /** ********* Raft Quorum Configuration *********/ + public static final List QUORUM_VOTERS = RaftConfig.DEFAULT_QUORUM_VOTERS; + public static final int QUORUM_ELECTION_TIMEOUT_MS = RaftConfig.DEFAULT_QUORUM_ELECTION_TIMEOUT_MS; + public static final int QUORUM_FETCH_TIMEOUT_MS = RaftConfig.DEFAULT_QUORUM_FETCH_TIMEOUT_MS; + public static final int QUORUM_ELECTION_BACKOFF_MS = RaftConfig.DEFAULT_QUORUM_ELECTION_BACKOFF_MAX_MS; + public static final int QUORUM_LINGER_MS = RaftConfig.DEFAULT_QUORUM_LINGER_MS; + public static final int QUORUM_REQUEST_TIMEOUT_MS = RaftConfig.DEFAULT_QUORUM_REQUEST_TIMEOUT_MS; + public static final int QUORUM_RETRY_BACKOFF_MS = RaftConfig.DEFAULT_QUORUM_RETRY_BACKOFF_MS; +} diff --git a/settings.gradle b/settings.gradle index d3a1cba454b..8a73d2ef57f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -97,6 +97,7 @@ include 'clients', 'streams:upgrade-system-tests-35', 'tools', 'tools:tools-api', + 'transaction-coordinator', 'trogdor' project(":storage:api").name = "storage-api" diff --git a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionLogConfig.java b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionLogConfig.java new file mode 100644 index 00000000000..00b7c9ef030 --- /dev/null +++ b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionLogConfig.java @@ -0,0 +1,26 @@ +/* + * 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.coordinator.transaction; + +public class TransactionLogConfig { + // Log-level config default values + public static final int DEFAULT_NUM_PARTITIONS = 50; + public static final int DEFAULT_SEGMENT_BYTES = 100 * 1024 * 1024; + public static final short DEFAULT_REPLICATION_FACTOR = 3; + public static final int DEFAULT_MIN_IN_SYNC_REPLICAS = 2; + public static final int DEFAULT_LOAD_BUFFER_SIZE = 5 * 1024 * 1024; +} diff --git a/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionStateManagerConfig.java b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionStateManagerConfig.java new file mode 100644 index 00000000000..b4889607a72 --- /dev/null +++ b/transaction-coordinator/src/main/java/org/apache/kafka/coordinator/transaction/TransactionStateManagerConfig.java @@ -0,0 +1,30 @@ +/* + * 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.coordinator.transaction; + +import java.util.concurrent.TimeUnit; + +public class TransactionStateManagerConfig { + // Default transaction management config values + public static final int DEFAULT_TRANSACTIONS_MAX_TIMEOUT_MS = (int) TimeUnit.MINUTES.toMillis(15); + public static final int DEFAULT_TRANSACTIONAL_ID_EXPIRATION_MS = (int) TimeUnit.DAYS.toMillis(7); + public static final int DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS = (int) TimeUnit.SECONDS.toMillis(10); + public static final int DEFAULT_REMOVE_EXPIRED_TRANSACTIONAL_IDS_INTERVAL_MS = (int) TimeUnit.HOURS.toMillis(1); + + public static final String METRICS_GROUP = "transaction-coordinator-metrics"; + public static final String LOAD_TIME_SENSOR = "TransactionsPartitionLoadTime"; +}