KAFKA-15853: Move KafkaConfig.Defaults to server module (#15158)

Reviewers: Mickael Maison <mickael.maison@gmail.com>, Ismael Juma <ismael@juma.me.uk>
, David Jacot <djacot@confluent.io>, Nikolay <NIzhikov@gmail.com>
This commit is contained in:
Omnia Ibrahim 2024-01-22 09:29:11 -05:00 committed by GitHub
parent cf90382fb9
commit 62ce551826
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
39 changed files with 692 additions and 559 deletions

View File

@ -849,6 +849,9 @@ project(':server') {
dependencies { dependencies {
implementation project(':clients') implementation project(':clients')
implementation project(':server-common') implementation project(':server-common')
implementation project(':group-coordinator')
implementation project(':transaction-coordinator')
implementation project(':raft')
implementation libs.metrics implementation libs.metrics
implementation libs.slf4jApi implementation libs.slf4jApi
@ -926,6 +929,7 @@ project(':core') {
implementation project(':server-common') implementation project(':server-common')
implementation project(':group-coordinator') implementation project(':group-coordinator')
implementation project(':transaction-coordinator')
implementation project(':metadata') implementation project(':metadata')
implementation project(':storage:storage-api') implementation project(':storage:storage-api')
implementation project(':tools:tools-api') implementation project(':tools:tools-api')
@ -1363,6 +1367,27 @@ project(':group-coordinator') {
srcJar.dependsOn 'processMessages' 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') { project(':examples') {
archivesBaseName = "kafka-examples" archivesBaseName = "kafka-examples"

View File

@ -70,6 +70,11 @@
<!-- server-metrics specific classes --> <!-- server-metrics specific classes -->
<allow pkg="org.apache.kafka.server.metrics" /> <allow pkg="org.apache.kafka.server.metrics" />
<!-- server-configs dependencies -->
<allow pkg="org.apache.kafka.coordinator" />
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.raft" />
<subpackage name="metrics"> <subpackage name="metrics">
<allow class="org.apache.kafka.server.authorizer.AuthorizableRequestContext" /> <allow class="org.apache.kafka.server.authorizer.AuthorizableRequestContext" />
<allow pkg="org.apache.kafka.server.telemetry" /> <allow pkg="org.apache.kafka.server.telemetry" />

View File

@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
import java.util.{Collections, Properties} import java.util.{Collections, Properties}
import joptsimple._ import joptsimple._
import kafka.server.DynamicConfig.QuotaConfigs 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.{Exit, Logging, PasswordEncoder}
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import kafka.zk.{AdminZkClient, KafkaZkClient} 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.JaasUtils
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism} import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism}
import org.apache.kafka.common.utils.{Sanitizer, Time, Utils} 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.server.util.{CommandDefaultOptions, CommandLineUtils}
import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.storage.internals.log.LogConfig
import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.client.ZKClientConfig
@ -216,9 +216,9 @@ object ConfigCommand extends Logging {
throw new IllegalArgumentException("Password encoder secret not specified")) throw new IllegalArgumentException("Password encoder secret not specified"))
PasswordEncoder.encrypting(new Password(encoderSecret), PasswordEncoder.encrypting(new Password(encoderSecret),
None, None,
encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderCipherAlgorithmProp, Defaults.PasswordEncoderCipherAlgorithm), encoderConfigs.getOrElse(KafkaConfig.PasswordEncoderCipherAlgorithmProp, Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM),
encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderKeyLength), encoderConfigs.get(KafkaConfig.PasswordEncoderKeyLengthProp).map(_.toInt).getOrElse(Defaults.PASSWORD_ENCODER_KEY_LENGTH),
encoderConfigs.get(KafkaConfig.PasswordEncoderIterationsProp).map(_.toInt).getOrElse(Defaults.PasswordEncoderIterations)) encoderConfigs.get(KafkaConfig.PasswordEncoderIterationsProp).map(_.toInt).getOrElse(Defaults.PASSWORD_ENCODER_ITERATIONS))
} }
/** /**

View File

@ -38,13 +38,6 @@ import scala.jdk.CollectionConverters._
*/ */
object TransactionLog { 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 // enforce always using
// 1. cleanup policy = compact // 1. cleanup policy = compact
// 2. compression = none // 2. compression = none

View File

@ -18,10 +18,9 @@ package kafka.coordinator.transaction
import java.nio.ByteBuffer import java.nio.ByteBuffer
import java.util.Properties import java.util.Properties
import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantReadWriteLock 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.CoreUtils.{inReadLock, inWriteLock}
import kafka.utils.{Logging, Pool} import kafka.utils.{Logging, Pool}
import kafka.utils.Implicits._ 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.requests.TransactionResult
import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.utils.{Time, Utils}
import org.apache.kafka.common.{KafkaException, TopicPartition} 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.record.BrokerCompressionType
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation}
@ -44,17 +45,6 @@ import scala.jdk.CollectionConverters._
import scala.collection.mutable 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: * Transaction state manager is part of the transaction coordinator, it manages:
* *
@ -100,13 +90,13 @@ class TransactionStateManager(brokerId: Int,
@volatile private var transactionTopicPartitionCount: Int = _ @volatile private var transactionTopicPartitionCount: Int = _
/** setup metrics*/ /** 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", 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()) "The max time it took to load the partitions in the last 30sec"), new Max())
partitionLoadSensor.add(metrics.metricName("partition-load-time-avg", 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()) "The avg time it took to load the partitions in the last 30sec"), new Avg())
// visible for testing only // visible for testing only
@ -808,16 +798,16 @@ private[transaction] case class TxnMetadataCacheEntry(coordinatorEpoch: Int,
private[transaction] case class CoordinatorEpochAndTxnMetadata(coordinatorEpoch: Int, private[transaction] case class CoordinatorEpochAndTxnMetadata(coordinatorEpoch: Int,
transactionMetadata: TransactionMetadata) transactionMetadata: TransactionMetadata)
private[transaction] case class TransactionConfig(transactionalIdExpirationMs: Int = TransactionStateManager.DefaultTransactionalIdExpirationMs, private[transaction] case class TransactionConfig(transactionalIdExpirationMs: Int = TransactionStateManagerConfig.DEFAULT_TRANSACTIONAL_ID_EXPIRATION_MS,
transactionMaxTimeoutMs: Int = TransactionStateManager.DefaultTransactionsMaxTimeoutMs, transactionMaxTimeoutMs: Int = TransactionStateManagerConfig.DEFAULT_TRANSACTIONS_MAX_TIMEOUT_MS,
transactionLogNumPartitions: Int = TransactionLog.DefaultNumPartitions, transactionLogNumPartitions: Int = TransactionLogConfig.DEFAULT_NUM_PARTITIONS,
transactionLogReplicationFactor: Short = TransactionLog.DefaultReplicationFactor, transactionLogReplicationFactor: Short = TransactionLogConfig.DEFAULT_REPLICATION_FACTOR,
transactionLogSegmentBytes: Int = TransactionLog.DefaultSegmentBytes, transactionLogSegmentBytes: Int = TransactionLogConfig.DEFAULT_SEGMENT_BYTES,
transactionLogLoadBufferSize: Int = TransactionLog.DefaultLoadBufferSize, transactionLogLoadBufferSize: Int = TransactionLogConfig.DEFAULT_LOAD_BUFFER_SIZE,
transactionLogMinInsyncReplicas: Int = TransactionLog.DefaultMinInSyncReplicas, transactionLogMinInsyncReplicas: Int = TransactionLogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS,
abortTimedOutTransactionsIntervalMs: Int = TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs, abortTimedOutTransactionsIntervalMs: Int = TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS,
removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManager.DefaultRemoveExpiredTransactionalIdsIntervalMs, removeExpiredTransactionalIdsIntervalMs: Int = TransactionStateManagerConfig.DEFAULT_REMOVE_EXPIRED_TRANSACTIONAL_IDS_INTERVAL_MS,
requestTimeoutMs: Int = Defaults.RequestTimeoutMs) requestTimeoutMs: Int = Defaults.REQUEST_TIMEOUT_MS)
case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) { case class TransactionalIdAndProducerIdEpoch(transactionalId: String, producerId: Long, producerEpoch: Short) {
override def toString: String = { override def toString: String = {

View File

@ -20,8 +20,10 @@
package kafka.metrics package kafka.metrics
import kafka.server.{Defaults, KafkaConfig} import kafka.server.KafkaConfig
import kafka.utils.{CoreUtils, VerifiableProperties} import kafka.utils.{CoreUtils, VerifiableProperties}
import org.apache.kafka.server.config.Defaults
import scala.collection.Seq import scala.collection.Seq
class KafkaMetricsConfig(props: VerifiableProperties) { class KafkaMetricsConfig(props: VerifiableProperties) {
@ -31,11 +33,11 @@ class KafkaMetricsConfig(props: VerifiableProperties) {
* classpath and will be instantiated at run-time. * classpath and will be instantiated at run-time.
*/ */
val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(KafkaConfig.KafkaMetricsReporterClassesProp, val reporters: Seq[String] = CoreUtils.parseCsvList(props.getString(KafkaConfig.KafkaMetricsReporterClassesProp,
Defaults.KafkaMetricReporterClasses)) Defaults.KAFKA_METRIC_REPORTER_CLASSES))
/** /**
* The metrics polling interval (in seconds). * The metrics polling interval (in seconds).
*/ */
val pollingIntervalSecs: Int = props.getInt(KafkaConfig.KafkaMetricsPollingIntervalSecondsProp, val pollingIntervalSecs: Int = props.getInt(KafkaConfig.KafkaMetricsPollingIntervalSecondsProp,
Defaults.KafkaMetricsPollingIntervalSeconds) Defaults.KAFKA_METRICS_POLLING_INTERVAL_SECONDS)
} }

View File

@ -19,9 +19,8 @@ package kafka.server
import java.{lang, util} import java.{lang, util}
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.{Collections, Locale, Properties} import java.util.{Collections, Properties}
import kafka.cluster.EndPoint import kafka.cluster.EndPoint
import kafka.coordinator.transaction.{TransactionLog, TransactionStateManager}
import kafka.security.authorizer.AuthorizerUtils import kafka.security.authorizer.AuthorizerUtils
import kafka.server.KafkaConfig.{ControllerListenerNamesProp, ListenerSecurityProtocolMapProp} import kafka.server.KafkaConfig.{ControllerListenerNamesProp, ListenerSecurityProtocolMapProp}
import kafka.utils.CoreUtils.parseCsvList import kafka.utils.CoreUtils.parseCsvList
@ -29,26 +28,23 @@ import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.common.Reconfigurable 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.ConfigDef.{ConfigKey, ValidList}
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.types.Password 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.network.ListenerName
import org.apache.kafka.common.record.{CompressionType, LegacyRecord, Records, TimestampType} 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.KafkaPrincipalSerde
import org.apache.kafka.common.security.auth.SecurityProtocol 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.common.utils.Utils
import org.apache.kafka.coordinator.group.Group.GroupType import org.apache.kafka.coordinator.group.Group.GroupType
import org.apache.kafka.coordinator.group.OffsetConfig import org.apache.kafka.coordinator.group.assignor.PartitionAssignor
import org.apache.kafka.coordinator.group.assignor.{PartitionAssignor, RangeAssignor, UniformAssignor}
import org.apache.kafka.raft.RaftConfig import org.apache.kafka.raft.RaftConfig
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.authorizer.Authorizer import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator} import org.apache.kafka.server.common.{MetadataVersion, MetadataVersionValidator}
import org.apache.kafka.server.common.MetadataVersion._ 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, ProducerStateManagerConfig}
import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
@ -60,241 +56,6 @@ import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq} 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 { object KafkaConfig {
private val LogConfigPrefix = "log." 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." "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 " + 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 " + "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 <code>$MetadataSnapshotMaxIntervalMsProp</code> configuration. The Kafka node will generate a snapshot when " + s"the <code>$MetadataSnapshotMaxIntervalMsProp</code> configuration. The Kafka node will generate a snapshot when " +
"either the maximum time interval is reached or the maximum bytes limit is reached." "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 " + 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 " + "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 <code>$MetadataSnapshotMaxNewRecordBytesProp</code> " + s"snapshots based on the number of metadata bytes, see the <code>$MetadataSnapshotMaxNewRecordBytesProp</code> " +
"configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " + "configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " +
"maximum bytes limit is reached." "maximum bytes limit is reached."
val MetadataMaxIdleIntervalMsDoc = "This configuration controls how often the active " + 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 " + "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 " + 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 " + "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." "Note: The ZooKeeper-based controller should not set this configuration."
@ -1204,11 +965,11 @@ object KafkaConfig {
/** ********* Zookeeper Configuration ***********/ /** ********* Zookeeper Configuration ***********/
.define(ZkConnectProp, STRING, null, HIGH, ZkConnectDoc) .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(ZkConnectionTimeoutMsProp, INT, null, HIGH, ZkConnectionTimeoutMsDoc)
.define(ZkEnableSecureAclsProp, BOOLEAN, Defaults.ZkEnableSecureAcls, HIGH, ZkEnableSecureAclsDoc) .define(ZkEnableSecureAclsProp, BOOLEAN, Defaults.ZK_ENABLE_SECURE_ACLS, HIGH, ZkEnableSecureAclsDoc)
.define(ZkMaxInFlightRequestsProp, INT, Defaults.ZkMaxInFlightRequests, atLeast(1), HIGH, ZkMaxInFlightRequestsDoc) .define(ZkMaxInFlightRequestsProp, INT, Defaults.ZK_MAX_IN_FLIGHT_REQUESTS, atLeast(1), HIGH, ZkMaxInFlightRequestsDoc)
.define(ZkSslClientEnableProp, BOOLEAN, Defaults.ZkSslClientEnable, MEDIUM, ZkSslClientEnableDoc) .define(ZkSslClientEnableProp, BOOLEAN, Defaults.ZK_SSL_CLIENT_ENABLE, MEDIUM, ZkSslClientEnableDoc)
.define(ZkClientCnxnSocketProp, STRING, null, MEDIUM, ZkClientCnxnSocketDoc) .define(ZkClientCnxnSocketProp, STRING, null, MEDIUM, ZkClientCnxnSocketDoc)
.define(ZkSslKeyStoreLocationProp, STRING, null, MEDIUM, ZkSslKeyStoreLocationDoc) .define(ZkSslKeyStoreLocationProp, STRING, null, MEDIUM, ZkSslKeyStoreLocationDoc)
.define(ZkSslKeyStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslKeyStorePasswordDoc) .define(ZkSslKeyStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslKeyStorePasswordDoc)
@ -1216,77 +977,77 @@ object KafkaConfig {
.define(ZkSslTrustStoreLocationProp, STRING, null, MEDIUM, ZkSslTrustStoreLocationDoc) .define(ZkSslTrustStoreLocationProp, STRING, null, MEDIUM, ZkSslTrustStoreLocationDoc)
.define(ZkSslTrustStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslTrustStorePasswordDoc) .define(ZkSslTrustStorePasswordProp, PASSWORD, null, MEDIUM, ZkSslTrustStorePasswordDoc)
.define(ZkSslTrustStoreTypeProp, STRING, null, MEDIUM, ZkSslTrustStoreTypeDoc) .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(ZkSslEnabledProtocolsProp, LIST, null, LOW, ZkSslEnabledProtocolsDoc)
.define(ZkSslCipherSuitesProp, LIST, null, LOW, ZkSslCipherSuitesDoc) .define(ZkSslCipherSuitesProp, LIST, null, LOW, ZkSslCipherSuitesDoc)
.define(ZkSslEndpointIdentificationAlgorithmProp, STRING, Defaults.ZkSslEndpointIdentificationAlgorithm, LOW, ZkSslEndpointIdentificationAlgorithmDoc) .define(ZkSslEndpointIdentificationAlgorithmProp, STRING, Defaults.ZK_SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, ZkSslEndpointIdentificationAlgorithmDoc)
.define(ZkSslCrlEnableProp, BOOLEAN, Defaults.ZkSslCrlEnable, LOW, ZkSslCrlEnableDoc) .define(ZkSslCrlEnableProp, BOOLEAN, Defaults.ZK_SSL_CRL_ENABLE, LOW, ZkSslCrlEnableDoc)
.define(ZkSslOcspEnableProp, BOOLEAN, Defaults.ZkSslOcspEnable, LOW, ZkSslOcspEnableDoc) .define(ZkSslOcspEnableProp, BOOLEAN, Defaults.ZK_SSL_OCSP_ENABLE, LOW, ZkSslOcspEnableDoc)
/** ********* General Configuration ***********/ /** ********* General Configuration ***********/
.define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BrokerIdGenerationEnable, MEDIUM, BrokerIdGenerationEnableDoc) .define(BrokerIdGenerationEnableProp, BOOLEAN, Defaults.BROKER_ID_GENERATION_ENABLE, MEDIUM, BrokerIdGenerationEnableDoc)
.define(MaxReservedBrokerIdProp, INT, Defaults.MaxReservedBrokerId, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc) .define(MaxReservedBrokerIdProp, INT, Defaults.MAX_RESERVED_BROKER_ID, atLeast(0), MEDIUM, MaxReservedBrokerIdDoc)
.define(BrokerIdProp, INT, Defaults.BrokerId, HIGH, BrokerIdDoc) .define(BrokerIdProp, INT, Defaults.BROKER_ID, HIGH, BrokerIdDoc)
.define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc) .define(MessageMaxBytesProp, INT, LogConfig.DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), HIGH, MessageMaxBytesDoc)
.define(NumNetworkThreadsProp, INT, Defaults.NumNetworkThreads, atLeast(1), HIGH, NumNetworkThreadsDoc) .define(NumNetworkThreadsProp, INT, Defaults.NUM_NETWORK_THREADS, atLeast(1), HIGH, NumNetworkThreadsDoc)
.define(NumIoThreadsProp, INT, Defaults.NumIoThreads, atLeast(1), HIGH, NumIoThreadsDoc) .define(NumIoThreadsProp, INT, Defaults.NUM_IO_THREADS, atLeast(1), HIGH, NumIoThreadsDoc)
.define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc) .define(NumReplicaAlterLogDirsThreadsProp, INT, null, HIGH, NumReplicaAlterLogDirsThreadsDoc)
.define(BackgroundThreadsProp, INT, Defaults.BackgroundThreads, atLeast(1), HIGH, BackgroundThreadsDoc) .define(BackgroundThreadsProp, INT, Defaults.BACKGROUND_THREADS, atLeast(1), HIGH, BackgroundThreadsDoc)
.define(QueuedMaxRequestsProp, INT, Defaults.QueuedMaxRequests, atLeast(1), HIGH, QueuedMaxRequestsDoc) .define(QueuedMaxRequestsProp, INT, Defaults.QUEUED_MAX_REQUESTS, atLeast(1), HIGH, QueuedMaxRequestsDoc)
.define(QueuedMaxBytesProp, LONG, Defaults.QueuedMaxRequestBytes, MEDIUM, QueuedMaxRequestBytesDoc) .define(QueuedMaxBytesProp, LONG, Defaults.QUEUED_MAX_REQUEST_BYTES, MEDIUM, QueuedMaxRequestBytesDoc)
.define(RequestTimeoutMsProp, INT, Defaults.RequestTimeoutMs, HIGH, RequestTimeoutMsDoc) .define(RequestTimeoutMsProp, INT, Defaults.REQUEST_TIMEOUT_MS, HIGH, RequestTimeoutMsDoc)
.define(ConnectionSetupTimeoutMsProp, LONG, Defaults.ConnectionSetupTimeoutMs, MEDIUM, ConnectionSetupTimeoutMsDoc) .define(ConnectionSetupTimeoutMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MS, MEDIUM, ConnectionSetupTimeoutMsDoc)
.define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.ConnectionSetupTimeoutMaxMs, MEDIUM, ConnectionSetupTimeoutMaxMsDoc) .define(ConnectionSetupTimeoutMaxMsProp, LONG, Defaults.CONNECTION_SETUP_TIMEOUT_MAX_MS, MEDIUM, ConnectionSetupTimeoutMaxMsDoc)
/* /*
* KRaft mode configs. * KRaft mode configs.
*/ */
.define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.MetadataSnapshotMaxNewRecordBytes, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc) .define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc)
.define(MetadataSnapshotMaxIntervalMsProp, LONG, Defaults.MetadataSnapshotMaxIntervalMs, atLeast(0), HIGH, MetadataSnapshotMaxIntervalMsDoc) .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(ProcessRolesProp, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, ProcessRolesDoc)
.define(NodeIdProp, INT, Defaults.EmptyNodeId, null, HIGH, NodeIdDoc) .define(NodeIdProp, INT, Defaults.EMPTY_NODE_ID, null, HIGH, NodeIdDoc)
.define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.InitialBrokerRegistrationTimeoutMs, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc) .define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc)
.define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BrokerHeartbeatIntervalMs, null, MEDIUM, BrokerHeartbeatIntervalMsDoc) .define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BROKER_HEARTBEAT_INTERVAL_MS, null, MEDIUM, BrokerHeartbeatIntervalMsDoc)
.define(BrokerSessionTimeoutMsProp, INT, Defaults.BrokerSessionTimeoutMs, null, MEDIUM, BrokerSessionTimeoutMsDoc) .define(BrokerSessionTimeoutMsProp, INT, Defaults.BROKER_SESSION_TIMEOUT_MS, null, MEDIUM, BrokerSessionTimeoutMsDoc)
.define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc) .define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc)
.define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc) .define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc)
.define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc) .define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc)
.define(MetadataLogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc) .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) .defineInternal(MetadataLogSegmentMinBytesProp, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentMinBytesDoc)
.define(MetadataLogSegmentMillisProp, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, MetadataLogSegmentMillisDoc) .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(MetadataMaxRetentionMillisProp, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, MetadataMaxRetentionMillisDoc)
.define(MetadataMaxIdleIntervalMsProp, INT, Defaults.MetadataMaxIdleIntervalMs, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) .define(MetadataMaxIdleIntervalMsProp, INT, Defaults.METADATA_MAX_IDLE_INTERVAL_MS, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc)
.defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.ServerMaxStartupTimeMs, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc)
.define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") .define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration")
.define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas") .define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas")
/************* Authorizer Configuration ***********/ /************* 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) .define(EarlyStartListenersProp, STRING, null, HIGH, EarlyStartListenersDoc)
/** ********* Socket Server Configuration ***********/ /** ********* Socket Server Configuration ***********/
.define(ListenersProp, STRING, Defaults.Listeners, HIGH, ListenersDoc) .define(ListenersProp, STRING, Defaults.LISTENERS, HIGH, ListenersDoc)
.define(AdvertisedListenersProp, STRING, null, HIGH, AdvertisedListenersDoc) .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(ControlPlaneListenerNameProp, STRING, null, HIGH, controlPlaneListenerNameDoc)
.define(SocketSendBufferBytesProp, INT, Defaults.SocketSendBufferBytes, HIGH, SocketSendBufferBytesDoc) .define(SocketSendBufferBytesProp, INT, Defaults.SOCKET_SEND_BUFFER_BYTES, HIGH, SocketSendBufferBytesDoc)
.define(SocketReceiveBufferBytesProp, INT, Defaults.SocketReceiveBufferBytes, HIGH, SocketReceiveBufferBytesDoc) .define(SocketReceiveBufferBytesProp, INT, Defaults.SOCKET_RECEIVE_BUFFER_BYTES, HIGH, SocketReceiveBufferBytesDoc)
.define(SocketRequestMaxBytesProp, INT, Defaults.SocketRequestMaxBytes, atLeast(1), HIGH, SocketRequestMaxBytesDoc) .define(SocketRequestMaxBytesProp, INT, Defaults.SOCKET_REQUEST_MAX_BYTES, atLeast(1), HIGH, SocketRequestMaxBytesDoc)
.define(SocketListenBacklogSizeProp, INT, Defaults.SocketListenBacklogSize, atLeast(1), MEDIUM, SocketListenBacklogSizeDoc) .define(SocketListenBacklogSizeProp, INT, Defaults.SOCKET_LISTEN_BACKLOG_SIZE, atLeast(1), MEDIUM, SocketListenBacklogSizeDoc)
.define(MaxConnectionsPerIpProp, INT, Defaults.MaxConnectionsPerIp, atLeast(0), MEDIUM, MaxConnectionsPerIpDoc) .define(MaxConnectionsPerIpProp, INT, Defaults.MAX_CONNECTIONS_PER_IP, atLeast(0), MEDIUM, MaxConnectionsPerIpDoc)
.define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MaxConnectionsPerIpOverrides, MEDIUM, MaxConnectionsPerIpOverridesDoc) .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MAX_CONNECTIONS_PER_IP_OVERRIDES, MEDIUM, MaxConnectionsPerIpOverridesDoc)
.define(MaxConnectionsProp, INT, Defaults.MaxConnections, atLeast(0), MEDIUM, MaxConnectionsDoc) .define(MaxConnectionsProp, INT, Defaults.MAX_CONNECTIONS, atLeast(0), MEDIUM, MaxConnectionsDoc)
.define(MaxConnectionCreationRateProp, INT, Defaults.MaxConnectionCreationRate, atLeast(0), MEDIUM, MaxConnectionCreationRateDoc) .define(MaxConnectionCreationRateProp, INT, Defaults.MAX_CONNECTION_CREATION_RATE, atLeast(0), MEDIUM, MaxConnectionCreationRateDoc)
.define(ConnectionsMaxIdleMsProp, LONG, Defaults.ConnectionsMaxIdleMs, MEDIUM, ConnectionsMaxIdleMsDoc) .define(ConnectionsMaxIdleMsProp, LONG, Defaults.CONNECTIONS_MAX_IDLE_MS, MEDIUM, ConnectionsMaxIdleMsDoc)
.define(FailedAuthenticationDelayMsProp, INT, Defaults.FailedAuthenticationDelayMs, atLeast(0), LOW, FailedAuthenticationDelayMsDoc) .define(FailedAuthenticationDelayMsProp, INT, Defaults.FAILED_AUTHENTICATION_DELAY_MS, atLeast(0), LOW, FailedAuthenticationDelayMsDoc)
/************ Rack Configuration ******************/ /************ Rack Configuration ******************/
.define(RackProp, STRING, null, MEDIUM, RackDoc) .define(RackProp, STRING, null, MEDIUM, RackDoc)
/** ********* Log Configuration ***********/ /** ********* Log Configuration ***********/
.define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) .define(NumPartitionsProp, INT, Defaults.NUM_PARTITIONS, atLeast(1), MEDIUM, NumPartitionsDoc)
.define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) .define(LogDirProp, STRING, Defaults.LOG_DIR, HIGH, LogDirDoc)
.define(LogDirsProp, STRING, null, HIGH, LogDirsDoc) .define(LogDirsProp, STRING, null, HIGH, LogDirsDoc)
.define(LogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), HIGH, LogSegmentBytesDoc) .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(LogRetentionTimeHoursProp, INT, TimeUnit.MILLISECONDS.toHours(LogConfig.DEFAULT_RETENTION_MS).toInt, HIGH, LogRetentionTimeHoursDoc)
.define(LogRetentionBytesProp, LONG, LogConfig.DEFAULT_RETENTION_BYTES, HIGH, LogRetentionBytesDoc) .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(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(LogCleanerThreadsProp, INT, Defaults.LOG_CLEANER_THREADS, atLeast(0), MEDIUM, LogCleanerThreadsDoc)
.define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LogCleanerIoMaxBytesPerSecond, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc) .define(LogCleanerIoMaxBytesPerSecondProp, DOUBLE, Defaults.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND, MEDIUM, LogCleanerIoMaxBytesPerSecondDoc)
.define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LogCleanerDedupeBufferSize, MEDIUM, LogCleanerDedupeBufferSizeDoc) .define(LogCleanerDedupeBufferSizeProp, LONG, Defaults.LOG_CLEANER_DEDUPE_BUFFER_SIZE, MEDIUM, LogCleanerDedupeBufferSizeDoc)
.define(LogCleanerIoBufferSizeProp, INT, Defaults.LogCleanerIoBufferSize, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc) .define(LogCleanerIoBufferSizeProp, INT, Defaults.LOG_CLEANER_IO_BUFFER_SIZE, atLeast(0), MEDIUM, LogCleanerIoBufferSizeDoc)
.define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LogCleanerDedupeBufferLoadFactor, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc) .define(LogCleanerDedupeBufferLoadFactorProp, DOUBLE, Defaults.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR, MEDIUM, LogCleanerDedupeBufferLoadFactorDoc)
.define(LogCleanerBackoffMsProp, LONG, Defaults.LogCleanerBackoffMs, atLeast(0), MEDIUM, LogCleanerBackoffMsDoc) .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(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(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(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) .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(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc)
.define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc) .define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc)
.define(LogFlushIntervalMsProp, LONG, null, HIGH, LogFlushIntervalMsDoc) .define(LogFlushIntervalMsProp, LONG, null, HIGH, LogFlushIntervalMsDoc)
.define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc) .define(LogFlushOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushOffsetCheckpointIntervalMsDoc)
.define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LogFlushStartOffsetCheckpointIntervalMs, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc) .define(LogFlushStartOffsetCheckpointIntervalMsProp, INT, Defaults.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS, atLeast(0), HIGH, LogFlushStartOffsetCheckpointIntervalMsDoc)
.define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc) .define(LogPreAllocateProp, BOOLEAN, LogConfig.DEFAULT_PREALLOCATE, MEDIUM, LogPreAllocateEnableDoc)
.define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NUM_RECOVERY_THREADS_PER_DATA_DIR, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc)
.define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AUTO_CREATE_TOPICS_ENABLE, HIGH, AutoCreateTopicsEnableDoc)
.define(MinInSyncReplicasProp, INT, LogConfig.DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), HIGH, MinInSyncReplicasDoc) .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(LogMessageFormatVersionProp, STRING, LogConfig.DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM, LogMessageFormatVersionDoc)
.define(LogMessageTimestampTypeProp, STRING, LogConfig.DEFAULT_MESSAGE_TIMESTAMP_TYPE, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) .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) .define(LogMessageDownConversionEnableProp, BOOLEAN, LogConfig.DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW, LogMessageDownConversionEnableDoc)
/** ********* Replication configuration ***********/ /** ********* Replication configuration ***********/
.define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) .define(ControllerSocketTimeoutMsProp, INT, Defaults.CONTROLLER_SOCKET_TIMEOUT_MS, MEDIUM, ControllerSocketTimeoutMsDoc)
.define(DefaultReplicationFactorProp, INT, Defaults.DefaultReplicationFactor, MEDIUM, DefaultReplicationFactorDoc) .define(DefaultReplicationFactorProp, INT, Defaults.REPLICATION_FACTOR, MEDIUM, DefaultReplicationFactorDoc)
.define(ReplicaLagTimeMaxMsProp, LONG, Defaults.ReplicaLagTimeMaxMs, HIGH, ReplicaLagTimeMaxMsDoc) .define(ReplicaLagTimeMaxMsProp, LONG, Defaults.REPLICA_LAG_TIME_MAX_MS, HIGH, ReplicaLagTimeMaxMsDoc)
.define(ReplicaSocketTimeoutMsProp, INT, Defaults.ReplicaSocketTimeoutMs, HIGH, ReplicaSocketTimeoutMsDoc) .define(ReplicaSocketTimeoutMsProp, INT, Defaults.REPLICA_SOCKET_TIMEOUT_MS, HIGH, ReplicaSocketTimeoutMsDoc)
.define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.ReplicaSocketReceiveBufferBytes, HIGH, ReplicaSocketReceiveBufferBytesDoc) .define(ReplicaSocketReceiveBufferBytesProp, INT, Defaults.REPLICA_SOCKET_RECEIVE_BUFFER_BYTES, HIGH, ReplicaSocketReceiveBufferBytesDoc)
.define(ReplicaFetchMaxBytesProp, INT, Defaults.ReplicaFetchMaxBytes, atLeast(0), MEDIUM, ReplicaFetchMaxBytesDoc) .define(ReplicaFetchMaxBytesProp, INT, Defaults.REPLICA_FETCH_MAX_BYTES, atLeast(0), MEDIUM, ReplicaFetchMaxBytesDoc)
.define(ReplicaFetchWaitMaxMsProp, INT, Defaults.ReplicaFetchWaitMaxMs, HIGH, ReplicaFetchWaitMaxMsDoc) .define(ReplicaFetchWaitMaxMsProp, INT, Defaults.REPLICA_FETCH_WAIT_MAX_MS, HIGH, ReplicaFetchWaitMaxMsDoc)
.define(ReplicaFetchBackoffMsProp, INT, Defaults.ReplicaFetchBackoffMs, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc) .define(ReplicaFetchBackoffMsProp, INT, Defaults.REPLICA_FETCH_BACKOFF_MS, atLeast(0), MEDIUM, ReplicaFetchBackoffMsDoc)
.define(ReplicaFetchMinBytesProp, INT, Defaults.ReplicaFetchMinBytes, HIGH, ReplicaFetchMinBytesDoc) .define(ReplicaFetchMinBytesProp, INT, Defaults.REPLICA_FETCH_MIN_BYTES, HIGH, ReplicaFetchMinBytesDoc)
.define(ReplicaFetchResponseMaxBytesProp, INT, Defaults.ReplicaFetchResponseMaxBytes, atLeast(0), MEDIUM, ReplicaFetchResponseMaxBytesDoc) .define(ReplicaFetchResponseMaxBytesProp, INT, Defaults.REPLICA_FETCH_RESPONSE_MAX_BYTES, atLeast(0), MEDIUM, ReplicaFetchResponseMaxBytesDoc)
.define(NumReplicaFetchersProp, INT, Defaults.NumReplicaFetchers, HIGH, NumReplicaFetchersDoc) .define(NumReplicaFetchersProp, INT, Defaults.NUM_REPLICA_FETCHERS, HIGH, NumReplicaFetchersDoc)
.define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.ReplicaHighWatermarkCheckpointIntervalMs, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc) .define(ReplicaHighWatermarkCheckpointIntervalMsProp, LONG, Defaults.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS, HIGH, ReplicaHighWatermarkCheckpointIntervalMsDoc)
.define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FetchPurgatoryPurgeIntervalRequests, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc) .define(FetchPurgatoryPurgeIntervalRequestsProp, INT, Defaults.FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, FetchPurgatoryPurgeIntervalRequestsDoc)
.define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.ProducerPurgatoryPurgeIntervalRequests, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc) .define(ProducerPurgatoryPurgeIntervalRequestsProp, INT, Defaults.PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, ProducerPurgatoryPurgeIntervalRequestsDoc)
.define(DeleteRecordsPurgatoryPurgeIntervalRequestsProp, INT, Defaults.DeleteRecordsPurgatoryPurgeIntervalRequests, MEDIUM, DeleteRecordsPurgatoryPurgeIntervalRequestsDoc) .define(DeleteRecordsPurgatoryPurgeIntervalRequestsProp, INT, Defaults.DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS, MEDIUM, DeleteRecordsPurgatoryPurgeIntervalRequestsDoc)
.define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AutoLeaderRebalanceEnable, HIGH, AutoLeaderRebalanceEnableDoc) .define(AutoLeaderRebalanceEnableProp, BOOLEAN, Defaults.AUTO_LEADER_REBALANCE_ENABLE, HIGH, AutoLeaderRebalanceEnableDoc)
.define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LeaderImbalancePerBrokerPercentage, HIGH, LeaderImbalancePerBrokerPercentageDoc) .define(LeaderImbalancePerBrokerPercentageProp, INT, Defaults.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE, HIGH, LeaderImbalancePerBrokerPercentageDoc)
.define(LeaderImbalanceCheckIntervalSecondsProp, LONG, Defaults.LeaderImbalanceCheckIntervalSeconds, atLeast(1), HIGH, LeaderImbalanceCheckIntervalSecondsDoc) .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(UncleanLeaderElectionEnableProp, BOOLEAN, LogConfig.DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE, HIGH, UncleanLeaderElectionEnableDoc)
.define(InterBrokerSecurityProtocolProp, STRING, Defaults.InterBrokerSecurityProtocol, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, InterBrokerSecurityProtocolDoc) .define(InterBrokerSecurityProtocolProp, STRING, Defaults.INTER_BROKER_SECURITY_PROTOCOL, in(Utils.enumOptions(classOf[SecurityProtocol]):_*), MEDIUM, InterBrokerSecurityProtocolDoc)
.define(InterBrokerProtocolVersionProp, STRING, Defaults.InterBrokerProtocolVersion, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc) .define(InterBrokerProtocolVersionProp, STRING, Defaults.INTER_BROKER_PROTOCOL_VERSION, new MetadataVersionValidator(), MEDIUM, InterBrokerProtocolVersionDoc)
.define(InterBrokerListenerNameProp, STRING, null, MEDIUM, InterBrokerListenerNameDoc) .define(InterBrokerListenerNameProp, STRING, null, MEDIUM, InterBrokerListenerNameDoc)
.define(ReplicaSelectorClassProp, STRING, null, MEDIUM, ReplicaSelectorClassDoc) .define(ReplicaSelectorClassProp, STRING, null, MEDIUM, ReplicaSelectorClassDoc)
/** ********* Controlled shutdown configuration ***********/ /** ********* Controlled shutdown configuration ***********/
.define(ControlledShutdownMaxRetriesProp, INT, Defaults.ControlledShutdownMaxRetries, MEDIUM, ControlledShutdownMaxRetriesDoc) .define(ControlledShutdownMaxRetriesProp, INT, Defaults.CONTROLLED_SHUTDOWN_MAX_RETRIES, MEDIUM, ControlledShutdownMaxRetriesDoc)
.define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS, MEDIUM, ControlledShutdownRetryBackoffMsDoc)
.define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.CONTROLLED_SHUTDOWN_ENABLE, MEDIUM, ControlledShutdownEnableDoc)
/** ********* Group coordinator configuration ***********/ /** ********* Group coordinator configuration ***********/
.define(GroupMinSessionTimeoutMsProp, INT, Defaults.GroupMinSessionTimeoutMs, MEDIUM, GroupMinSessionTimeoutMsDoc) .define(GroupMinSessionTimeoutMsProp, INT, Defaults.GROUP_MIN_SESSION_TIMEOUT_MS, MEDIUM, GroupMinSessionTimeoutMsDoc)
.define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GroupMaxSessionTimeoutMs, MEDIUM, GroupMaxSessionTimeoutMsDoc) .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GROUP_MAX_SESSION_TIMEOUT_MS, MEDIUM, GroupMaxSessionTimeoutMsDoc)
.define(GroupInitialRebalanceDelayMsProp, INT, Defaults.GroupInitialRebalanceDelayMs, MEDIUM, GroupInitialRebalanceDelayMsDoc) .define(GroupInitialRebalanceDelayMsProp, INT, Defaults.GROUP_INITIAL_REBALANCE_DELAY_MS, MEDIUM, GroupInitialRebalanceDelayMsDoc)
.define(GroupMaxSizeProp, INT, Defaults.GroupMaxSize, atLeast(1), MEDIUM, GroupMaxSizeDoc) .define(GroupMaxSizeProp, INT, Defaults.GROUP_MAX_SIZE, atLeast(1), MEDIUM, GroupMaxSizeDoc)
/** New group coordinator configs */ /** 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) 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. // 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 */ /** Consumer groups configs */
.define(ConsumerGroupSessionTimeoutMsProp, INT, Defaults.ConsumerGroupSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupSessionTimeoutMsDoc) .define(ConsumerGroupSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupSessionTimeoutMsDoc)
.define(ConsumerGroupMinSessionTimeoutMsProp, INT, Defaults.ConsumerGroupMinSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupMinSessionTimeoutMsDoc) .define(ConsumerGroupMinSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupMinSessionTimeoutMsDoc)
.define(ConsumerGroupMaxSessionTimeoutMsProp, INT, Defaults.ConsumerGroupMaxSessionTimeoutMs, atLeast(1), MEDIUM, ConsumerGroupMaxSessionTimeoutMsDoc) .define(ConsumerGroupMaxSessionTimeoutMsProp, INT, Defaults.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS, atLeast(1), MEDIUM, ConsumerGroupMaxSessionTimeoutMsDoc)
.define(ConsumerGroupHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupHeartbeatIntervalMsDoc) .define(ConsumerGroupHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupHeartbeatIntervalMsDoc)
.define(ConsumerGroupMinHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupMinHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupMinHeartbeatIntervalMsDoc) .define(ConsumerGroupMinHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupMinHeartbeatIntervalMsDoc)
.define(ConsumerGroupMaxHeartbeatIntervalMsProp, INT, Defaults.ConsumerGroupMaxHeartbeatIntervalMs, atLeast(1), MEDIUM, ConsumerGroupMaxHeartbeatIntervalMsDoc) .define(ConsumerGroupMaxHeartbeatIntervalMsProp, INT, Defaults.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS, atLeast(1), MEDIUM, ConsumerGroupMaxHeartbeatIntervalMsDoc)
.define(ConsumerGroupMaxSizeProp, INT, Defaults.ConsumerGroupMaxSize, atLeast(1), MEDIUM, ConsumerGroupMaxSizeDoc) .define(ConsumerGroupMaxSizeProp, INT, Defaults.CONSUMER_GROUP_MAX_SIZE, atLeast(1), MEDIUM, ConsumerGroupMaxSizeDoc)
.define(ConsumerGroupAssignorsProp, LIST, Defaults.ConsumerGroupAssignors, null, MEDIUM, ConsumerGroupAssignorsDoc) .define(ConsumerGroupAssignorsProp, LIST, Defaults.CONSUMER_GROUP_ASSIGNORS, null, MEDIUM, ConsumerGroupAssignorsDoc)
/** ********* Offset management configuration ***********/ /** ********* Offset management configuration ***********/
.define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) .define(OffsetMetadataMaxSizeProp, INT, Defaults.OFFSET_METADATA_MAX_SIZE, HIGH, OffsetMetadataMaxSizeDoc)
.define(OffsetsLoadBufferSizeProp, INT, Defaults.OffsetsLoadBufferSize, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc) .define(OffsetsLoadBufferSizeProp, INT, Defaults.OFFSETS_LOAD_BUFFER_SIZE, atLeast(1), HIGH, OffsetsLoadBufferSizeDoc)
.define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OffsetsTopicReplicationFactor, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc) .define(OffsetsTopicReplicationFactorProp, SHORT, Defaults.OFFSETS_TOPIC_REPLICATION_FACTOR, atLeast(1), HIGH, OffsetsTopicReplicationFactorDoc)
.define(OffsetsTopicPartitionsProp, INT, Defaults.OffsetsTopicPartitions, atLeast(1), HIGH, OffsetsTopicPartitionsDoc) .define(OffsetsTopicPartitionsProp, INT, Defaults.OFFSETS_TOPIC_PARTITIONS, atLeast(1), HIGH, OffsetsTopicPartitionsDoc)
.define(OffsetsTopicSegmentBytesProp, INT, Defaults.OffsetsTopicSegmentBytes, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc) .define(OffsetsTopicSegmentBytesProp, INT, Defaults.OFFSETS_TOPIC_SEGMENT_BYTES, atLeast(1), HIGH, OffsetsTopicSegmentBytesDoc)
.define(OffsetsTopicCompressionCodecProp, INT, Defaults.OffsetsTopicCompressionCodec, HIGH, OffsetsTopicCompressionCodecDoc) .define(OffsetsTopicCompressionCodecProp, INT, Defaults.OFFSETS_TOPIC_COMPRESSION_CODEC, HIGH, OffsetsTopicCompressionCodecDoc)
.define(OffsetsRetentionMinutesProp, INT, Defaults.OffsetsRetentionMinutes, atLeast(1), HIGH, OffsetsRetentionMinutesDoc) .define(OffsetsRetentionMinutesProp, INT, Defaults.OFFSETS_RETENTION_MINUTES, atLeast(1), HIGH, OffsetsRetentionMinutesDoc)
.define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OffsetsRetentionCheckIntervalMs, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc) .define(OffsetsRetentionCheckIntervalMsProp, LONG, Defaults.OFFSETS_RETENTION_CHECK_INTERVAL_MS, atLeast(1), HIGH, OffsetsRetentionCheckIntervalMsDoc)
.define(OffsetCommitTimeoutMsProp, INT, Defaults.OffsetCommitTimeoutMs, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc) .define(OffsetCommitTimeoutMsProp, INT, Defaults.OFFSET_COMMIT_TIMEOUT_MS, atLeast(1), HIGH, OffsetCommitTimeoutMsDoc)
.define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OffsetCommitRequiredAcks, HIGH, OffsetCommitRequiredAcksDoc) .define(OffsetCommitRequiredAcksProp, SHORT, Defaults.OFFSET_COMMIT_REQUIRED_ACKS, HIGH, OffsetCommitRequiredAcksDoc)
.define(DeleteTopicEnableProp, BOOLEAN, Defaults.DeleteTopicEnable, HIGH, DeleteTopicEnableDoc) .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc)
.define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc) .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc)
/** ********* Transaction management configuration ***********/ /** ********* Transaction management configuration ***********/
.define(TransactionalIdExpirationMsProp, INT, Defaults.TransactionalIdExpirationMs, atLeast(1), HIGH, TransactionalIdExpirationMsDoc) .define(TransactionalIdExpirationMsProp, INT, Defaults.TRANSACTIONAL_ID_EXPIRATION_MS, atLeast(1), HIGH, TransactionalIdExpirationMsDoc)
.define(TransactionsMaxTimeoutMsProp, INT, Defaults.TransactionsMaxTimeoutMs, atLeast(1), HIGH, TransactionsMaxTimeoutMsDoc) .define(TransactionsMaxTimeoutMsProp, INT, Defaults.TRANSACTIONS_MAX_TIMEOUT_MS, atLeast(1), HIGH, TransactionsMaxTimeoutMsDoc)
.define(TransactionsTopicMinISRProp, INT, Defaults.TransactionsTopicMinISR, atLeast(1), HIGH, TransactionsTopicMinISRDoc) .define(TransactionsTopicMinISRProp, INT, Defaults.TRANSACTIONS_TOPIC_MIN_ISR, atLeast(1), HIGH, TransactionsTopicMinISRDoc)
.define(TransactionsLoadBufferSizeProp, INT, Defaults.TransactionsLoadBufferSize, atLeast(1), HIGH, TransactionsLoadBufferSizeDoc) .define(TransactionsLoadBufferSizeProp, INT, Defaults.TRANSACTIONS_LOAD_BUFFER_SIZE, atLeast(1), HIGH, TransactionsLoadBufferSizeDoc)
.define(TransactionsTopicReplicationFactorProp, SHORT, Defaults.TransactionsTopicReplicationFactor, atLeast(1), HIGH, TransactionsTopicReplicationFactorDoc) .define(TransactionsTopicReplicationFactorProp, SHORT, Defaults.TRANSACTIONS_TOPIC_REPLICATION_FACTOR, atLeast(1), HIGH, TransactionsTopicReplicationFactorDoc)
.define(TransactionsTopicPartitionsProp, INT, Defaults.TransactionsTopicPartitions, atLeast(1), HIGH, TransactionsTopicPartitionsDoc) .define(TransactionsTopicPartitionsProp, INT, Defaults.TRANSACTIONS_TOPIC_PARTITIONS, atLeast(1), HIGH, TransactionsTopicPartitionsDoc)
.define(TransactionsTopicSegmentBytesProp, INT, Defaults.TransactionsTopicSegmentBytes, atLeast(1), HIGH, TransactionsTopicSegmentBytesDoc) .define(TransactionsTopicSegmentBytesProp, INT, Defaults.TRANSACTIONS_TOPIC_SEGMENT_BYTES, atLeast(1), HIGH, TransactionsTopicSegmentBytesDoc)
.define(TransactionsAbortTimedOutTransactionCleanupIntervalMsProp, INT, Defaults.TransactionsAbortTimedOutTransactionsCleanupIntervalMS, atLeast(1), LOW, TransactionsAbortTimedOutTransactionsIntervalMsDoc) .define(TransactionsAbortTimedOutTransactionCleanupIntervalMsProp, INT, Defaults.TRANSACTIONS_ABORT_TIMED_OUT_CLEANUP_INTERVAL_MS, atLeast(1), LOW, TransactionsAbortTimedOutTransactionsIntervalMsDoc)
.define(TransactionsRemoveExpiredTransactionalIdCleanupIntervalMsProp, INT, Defaults.TransactionsRemoveExpiredTransactionsCleanupIntervalMS, atLeast(1), LOW, TransactionsRemoveExpiredTransactionsIntervalMsDoc) .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 // 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 **************/ /** ********* Fetch Configuration **************/
.define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MaxIncrementalFetchSessionCacheSlots, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc) .define(MaxIncrementalFetchSessionCacheSlots, INT, Defaults.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS, atLeast(0), MEDIUM, MaxIncrementalFetchSessionCacheSlotsDoc)
.define(FetchMaxBytes, INT, Defaults.FetchMaxBytes, atLeast(1024), MEDIUM, FetchMaxBytesDoc) .define(FetchMaxBytes, INT, Defaults.FETCH_MAX_BYTES, atLeast(1024), MEDIUM, FetchMaxBytesDoc)
/** ********* Kafka Metrics Configuration ***********/ /** ********* Kafka Metrics Configuration ***********/
.define(MetricNumSamplesProp, INT, Defaults.MetricNumSamples, atLeast(1), LOW, MetricNumSamplesDoc) .define(MetricNumSamplesProp, INT, Defaults.METRIC_NUM_SAMPLES, atLeast(1), LOW, MetricNumSamplesDoc)
.define(MetricSampleWindowMsProp, LONG, Defaults.MetricSampleWindowMs, atLeast(1), LOW, MetricSampleWindowMsDoc) .define(MetricSampleWindowMsProp, LONG, Defaults.METRIC_SAMPLE_WINDOW_MS, atLeast(1), LOW, MetricSampleWindowMsDoc)
.define(MetricReporterClassesProp, LIST, Defaults.MetricReporterClasses, LOW, MetricReporterClassesDoc) .define(MetricReporterClassesProp, LIST, Defaults.METRIC_REPORTER_CLASSES, LOW, MetricReporterClassesDoc)
.define(MetricRecordingLevelProp, STRING, Defaults.MetricRecordingLevel, LOW, MetricRecordingLevelDoc) .define(MetricRecordingLevelProp, STRING, Defaults.METRIC_RECORDING_LEVEL, LOW, MetricRecordingLevelDoc)
.define(AutoIncludeJmxReporterProp, BOOLEAN, Defaults.AutoIncludeJmxReporter, LOW, AutoIncludeJmxReporterDoc) .define(AutoIncludeJmxReporterProp, BOOLEAN, Defaults.AUTO_INCLUDE_JMX_REPORTER, LOW, AutoIncludeJmxReporterDoc)
/** ********* Kafka Yammer Metrics Reporter Configuration for docs ***********/ /** ********* Kafka Yammer Metrics Reporter Configuration for docs ***********/
.define(KafkaMetricsReporterClassesProp, LIST, Defaults.KafkaMetricReporterClasses, LOW, KafkaMetricsReporterClassesDoc) .define(KafkaMetricsReporterClassesProp, LIST, Defaults.KAFKA_METRIC_REPORTER_CLASSES, LOW, KafkaMetricsReporterClassesDoc)
.define(KafkaMetricsPollingIntervalSecondsProp, INT, Defaults.KafkaMetricsPollingIntervalSeconds, atLeast(1), LOW, KafkaMetricsPollingIntervalSecondsDoc) .define(KafkaMetricsPollingIntervalSecondsProp, INT, Defaults.KAFKA_METRICS_POLLING_INTERVAL_SECONDS, atLeast(1), LOW, KafkaMetricsPollingIntervalSecondsDoc)
/** ********* Kafka Client Telemetry Metrics Configuration ***********/ /** ********* 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 ***********/ /** ********* Quota configuration ***********/
.define(NumQuotaSamplesProp, INT, Defaults.NumQuotaSamples, atLeast(1), LOW, NumQuotaSamplesDoc) .define(NumQuotaSamplesProp, INT, Defaults.NUM_QUOTA_SAMPLES, atLeast(1), LOW, NumQuotaSamplesDoc)
.define(NumReplicationQuotaSamplesProp, INT, Defaults.NumReplicationQuotaSamples, atLeast(1), LOW, NumReplicationQuotaSamplesDoc) .define(NumReplicationQuotaSamplesProp, INT, Defaults.NUM_REPLICATION_QUOTA_SAMPLES, atLeast(1), LOW, NumReplicationQuotaSamplesDoc)
.define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NumAlterLogDirsReplicationQuotaSamples, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc) .define(NumAlterLogDirsReplicationQuotaSamplesProp, INT, Defaults.NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES, atLeast(1), LOW, NumAlterLogDirsReplicationQuotaSamplesDoc)
.define(NumControllerQuotaSamplesProp, INT, Defaults.NumControllerQuotaSamples, atLeast(1), LOW, NumControllerQuotaSamplesDoc) .define(NumControllerQuotaSamplesProp, INT, Defaults.NUM_CONTROLLER_QUOTA_SAMPLES, atLeast(1), LOW, NumControllerQuotaSamplesDoc)
.define(QuotaWindowSizeSecondsProp, INT, Defaults.QuotaWindowSizeSeconds, atLeast(1), LOW, QuotaWindowSizeSecondsDoc) .define(QuotaWindowSizeSecondsProp, INT, Defaults.QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, QuotaWindowSizeSecondsDoc)
.define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc) .define(ReplicationQuotaWindowSizeSecondsProp, INT, Defaults.REPLICATION_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, ReplicationQuotaWindowSizeSecondsDoc)
.define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.AlterLogDirsReplicationQuotaWindowSizeSeconds, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc) .define(AlterLogDirsReplicationQuotaWindowSizeSecondsProp, INT, Defaults.ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, AlterLogDirsReplicationQuotaWindowSizeSecondsDoc)
.define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.ControllerQuotaWindowSizeSeconds, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc) .define(ControllerQuotaWindowSizeSecondsProp, INT, Defaults.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS, atLeast(1), LOW, ControllerQuotaWindowSizeSecondsDoc)
.define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc) .define(ClientQuotaCallbackClassProp, CLASS, null, LOW, ClientQuotaCallbackClassDoc)
/** ********* General Security Configuration ****************/ /** ********* General Security Configuration ****************/
.define(ConnectionsMaxReauthMsProp, LONG, Defaults.ConnectionsMaxReauthMsDefault, MEDIUM, ConnectionsMaxReauthMsDoc) .define(ConnectionsMaxReauthMsProp, LONG, Defaults.CONNECTIONS_MAX_REAUTH_MS, MEDIUM, ConnectionsMaxReauthMsDoc)
.define(SaslServerMaxReceiveSizeProp, INT, Defaults.DefaultServerMaxMaxReceiveSize, MEDIUM, SaslServerMaxReceiveSizeDoc) .define(SaslServerMaxReceiveSizeProp, INT, Defaults.SERVER_MAX_RECEIVE_SIZE, MEDIUM, SaslServerMaxReceiveSizeDoc)
.define(securityProviderClassProp, STRING, null, LOW, securityProviderClassDoc) .define(securityProviderClassProp, STRING, null, LOW, securityProviderClassDoc)
/** ********* SSL Configuration ****************/ /** ********* SSL Configuration ****************/
.define(PrincipalBuilderClassProp, CLASS, Defaults.DefaultPrincipalBuilder, MEDIUM, PrincipalBuilderClassDoc) .define(PrincipalBuilderClassProp, CLASS, Defaults.PRINCIPAL_BUILDER, MEDIUM, PrincipalBuilderClassDoc)
.define(SslProtocolProp, STRING, Defaults.SslProtocol, MEDIUM, SslProtocolDoc) .define(SslProtocolProp, STRING, Defaults.SSL_PROTOCOL, MEDIUM, SslProtocolDoc)
.define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc) .define(SslProviderProp, STRING, null, MEDIUM, SslProviderDoc)
.define(SslEnabledProtocolsProp, LIST, Defaults.SslEnabledProtocols, MEDIUM, SslEnabledProtocolsDoc) .define(SslEnabledProtocolsProp, LIST, Defaults.SSL_ENABLED_PROTOCOLS, MEDIUM, SslEnabledProtocolsDoc)
.define(SslKeystoreTypeProp, STRING, Defaults.SslKeystoreType, MEDIUM, SslKeystoreTypeDoc) .define(SslKeystoreTypeProp, STRING, Defaults.SSL_KEYSTORE_TYPE, MEDIUM, SslKeystoreTypeDoc)
.define(SslKeystoreLocationProp, STRING, null, MEDIUM, SslKeystoreLocationDoc) .define(SslKeystoreLocationProp, STRING, null, MEDIUM, SslKeystoreLocationDoc)
.define(SslKeystorePasswordProp, PASSWORD, null, MEDIUM, SslKeystorePasswordDoc) .define(SslKeystorePasswordProp, PASSWORD, null, MEDIUM, SslKeystorePasswordDoc)
.define(SslKeyPasswordProp, PASSWORD, null, MEDIUM, SslKeyPasswordDoc) .define(SslKeyPasswordProp, PASSWORD, null, MEDIUM, SslKeyPasswordDoc)
.define(SslKeystoreKeyProp, PASSWORD, null, MEDIUM, SslKeystoreKeyDoc) .define(SslKeystoreKeyProp, PASSWORD, null, MEDIUM, SslKeystoreKeyDoc)
.define(SslKeystoreCertificateChainProp, PASSWORD, null, MEDIUM, SslKeystoreCertificateChainDoc) .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(SslTruststoreLocationProp, STRING, null, MEDIUM, SslTruststoreLocationDoc)
.define(SslTruststorePasswordProp, PASSWORD, null, MEDIUM, SslTruststorePasswordDoc) .define(SslTruststorePasswordProp, PASSWORD, null, MEDIUM, SslTruststorePasswordDoc)
.define(SslTruststoreCertificatesProp, PASSWORD, null, MEDIUM, SslTruststoreCertificatesDoc) .define(SslTruststoreCertificatesProp, PASSWORD, null, MEDIUM, SslTruststoreCertificatesDoc)
.define(SslKeyManagerAlgorithmProp, STRING, Defaults.SslKeyManagerAlgorithm, MEDIUM, SslKeyManagerAlgorithmDoc) .define(SslKeyManagerAlgorithmProp, STRING, Defaults.SSL_KEY_MANAGER_ALGORITHM, MEDIUM, SslKeyManagerAlgorithmDoc)
.define(SslTrustManagerAlgorithmProp, STRING, Defaults.SslTrustManagerAlgorithm, MEDIUM, SslTrustManagerAlgorithmDoc) .define(SslTrustManagerAlgorithmProp, STRING, Defaults.SSL_TRUST_MANAGER_ALGORITHM, MEDIUM, SslTrustManagerAlgorithmDoc)
.define(SslEndpointIdentificationAlgorithmProp, STRING, Defaults.SslEndpointIdentificationAlgorithm, LOW, SslEndpointIdentificationAlgorithmDoc) .define(SslEndpointIdentificationAlgorithmProp, STRING, Defaults.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM, LOW, SslEndpointIdentificationAlgorithmDoc)
.define(SslSecureRandomImplementationProp, STRING, null, LOW, SslSecureRandomImplementationDoc) .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(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(SslEngineFactoryClassProp, CLASS, null, LOW, SslEngineFactoryClassDoc)
.define(SslAllowDnChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE, LOW, SslAllowDnChangesDoc) .define(SslAllowDnChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_DN_CHANGES_VALUE, LOW, SslAllowDnChangesDoc)
.define(SslAllowSanChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, SslAllowSanChangesDoc) .define(SslAllowSanChangesProp, BOOLEAN, BrokerSecurityConfigs.DEFAULT_SSL_ALLOW_SAN_CHANGES_VALUE, LOW, SslAllowSanChangesDoc)
/** ********* Sasl Configuration ****************/ /** ********* 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(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(SaslServerCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslServerCallbackHandlerClassDoc)
.define(SaslClientCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslClientCallbackHandlerClassDoc) .define(SaslClientCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslClientCallbackHandlerClassDoc)
.define(SaslLoginClassProp, CLASS, null, MEDIUM, SaslLoginClassDoc) .define(SaslLoginClassProp, CLASS, null, MEDIUM, SaslLoginClassDoc)
.define(SaslLoginCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslLoginCallbackHandlerClassDoc) .define(SaslLoginCallbackHandlerClassProp, CLASS, null, MEDIUM, SaslLoginCallbackHandlerClassDoc)
.define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, SaslKerberosServiceNameDoc) .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, SaslKerberosServiceNameDoc)
.define(SaslKerberosKinitCmdProp, STRING, Defaults.SaslKerberosKinitCmd, MEDIUM, SaslKerberosKinitCmdDoc) .define(SaslKerberosKinitCmdProp, STRING, Defaults.SASL_KERBEROS_KINIT_CMD, MEDIUM, SaslKerberosKinitCmdDoc)
.define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SaslKerberosTicketRenewWindowFactor, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc) .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc)
.define(SaslKerberosTicketRenewJitterProp, DOUBLE, Defaults.SaslKerberosTicketRenewJitter, MEDIUM, SaslKerberosTicketRenewJitterDoc) .define(SaslKerberosTicketRenewJitterProp, DOUBLE, Defaults.SASL_KERBEROS_TICKET_RENEW_JITTER, MEDIUM, SaslKerberosTicketRenewJitterDoc)
.define(SaslKerberosMinTimeBeforeReloginProp, LONG, Defaults.SaslKerberosMinTimeBeforeRelogin, MEDIUM, SaslKerberosMinTimeBeforeReloginDoc) .define(SaslKerberosMinTimeBeforeReloginProp, LONG, Defaults.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, MEDIUM, SaslKerberosMinTimeBeforeReloginDoc)
.define(SaslKerberosPrincipalToLocalRulesProp, LIST, Defaults.SaslKerberosPrincipalToLocalRules, MEDIUM, SaslKerberosPrincipalToLocalRulesDoc) .define(SaslKerberosPrincipalToLocalRulesProp, LIST, Defaults.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES, MEDIUM, SaslKerberosPrincipalToLocalRulesDoc)
.define(SaslLoginRefreshWindowFactorProp, DOUBLE, Defaults.SaslLoginRefreshWindowFactor, MEDIUM, SaslLoginRefreshWindowFactorDoc) .define(SaslLoginRefreshWindowFactorProp, DOUBLE, Defaults.SASL_LOGIN_REFRESH_WINDOW_FACTOR, MEDIUM, SaslLoginRefreshWindowFactorDoc)
.define(SaslLoginRefreshWindowJitterProp, DOUBLE, Defaults.SaslLoginRefreshWindowJitter, MEDIUM, SaslLoginRefreshWindowJitterDoc) .define(SaslLoginRefreshWindowJitterProp, DOUBLE, Defaults.SASL_LOGIN_REFRESH_WINDOW_JITTER, MEDIUM, SaslLoginRefreshWindowJitterDoc)
.define(SaslLoginRefreshMinPeriodSecondsProp, SHORT, Defaults.SaslLoginRefreshMinPeriodSeconds, MEDIUM, SaslLoginRefreshMinPeriodSecondsDoc) .define(SaslLoginRefreshMinPeriodSecondsProp, SHORT, Defaults.SASL_LOGIN_REFRESH_MIN_PERIOD_SECONDS, MEDIUM, SaslLoginRefreshMinPeriodSecondsDoc)
.define(SaslLoginRefreshBufferSecondsProp, SHORT, Defaults.SaslLoginRefreshBufferSeconds, MEDIUM, SaslLoginRefreshBufferSecondsDoc) .define(SaslLoginRefreshBufferSecondsProp, SHORT, Defaults.SASL_LOGIN_REFRESH_BUFFER_SECONDS, MEDIUM, SaslLoginRefreshBufferSecondsDoc)
.define(SaslLoginConnectTimeoutMsProp, INT, null, LOW, SaslLoginConnectTimeoutMsDoc) .define(SaslLoginConnectTimeoutMsProp, INT, null, LOW, SaslLoginConnectTimeoutMsDoc)
.define(SaslLoginReadTimeoutMsProp, INT, null, LOW, SaslLoginReadTimeoutMsDoc) .define(SaslLoginReadTimeoutMsProp, INT, null, LOW, SaslLoginReadTimeoutMsDoc)
.define(SaslLoginRetryBackoffMaxMsProp, LONG, Defaults.SaslLoginRetryBackoffMaxMs, LOW, SaslLoginRetryBackoffMaxMsDoc) .define(SaslLoginRetryBackoffMaxMsProp, LONG, Defaults.SASL_LOGIN_RETRY_BACKOFF_MAX_MS, LOW, SaslLoginRetryBackoffMaxMsDoc)
.define(SaslLoginRetryBackoffMsProp, LONG, Defaults.SaslLoginRetryBackoffMs, LOW, SaslLoginRetryBackoffMsDoc) .define(SaslLoginRetryBackoffMsProp, LONG, Defaults.SASL_LOGIN_RETRY_BACKOFF_MS, LOW, SaslLoginRetryBackoffMsDoc)
.define(SaslOAuthBearerScopeClaimNameProp, STRING, Defaults.SaslOAuthBearerScopeClaimName, LOW, SaslOAuthBearerScopeClaimNameDoc) .define(SaslOAuthBearerScopeClaimNameProp, STRING, Defaults.SASL_OAUTH_BEARER_SCOPE_CLAIM_NAME, LOW, SaslOAuthBearerScopeClaimNameDoc)
.define(SaslOAuthBearerSubClaimNameProp, STRING, Defaults.SaslOAuthBearerSubClaimName, LOW, SaslOAuthBearerSubClaimNameDoc) .define(SaslOAuthBearerSubClaimNameProp, STRING, Defaults.SASL_OAUTH_BEARER_SUB_CLAIM_NAME, LOW, SaslOAuthBearerSubClaimNameDoc)
.define(SaslOAuthBearerTokenEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerTokenEndpointUrlDoc) .define(SaslOAuthBearerTokenEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerTokenEndpointUrlDoc)
.define(SaslOAuthBearerJwksEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerJwksEndpointUrlDoc) .define(SaslOAuthBearerJwksEndpointUrlProp, STRING, null, MEDIUM, SaslOAuthBearerJwksEndpointUrlDoc)
.define(SaslOAuthBearerJwksEndpointRefreshMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRefreshMs, LOW, SaslOAuthBearerJwksEndpointRefreshMsDoc) .define(SaslOAuthBearerJwksEndpointRefreshMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_REFRESH_MS, LOW, SaslOAuthBearerJwksEndpointRefreshMsDoc)
.define(SaslOAuthBearerJwksEndpointRetryBackoffMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRetryBackoffMs, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMsDoc) .define(SaslOAuthBearerJwksEndpointRetryBackoffMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MS, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMsDoc)
.define(SaslOAuthBearerJwksEndpointRetryBackoffMaxMsProp, LONG, Defaults.SaslOAuthBearerJwksEndpointRetryBackoffMaxMs, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMaxMsDoc) .define(SaslOAuthBearerJwksEndpointRetryBackoffMaxMsProp, LONG, Defaults.SASL_OAUTH_BEARER_JWKS_ENDPOINT_RETRY_BACKOFF_MAX_MS, LOW, SaslOAuthBearerJwksEndpointRetryBackoffMaxMsDoc)
.define(SaslOAuthBearerClockSkewSecondsProp, INT, Defaults.SaslOAuthBearerClockSkewSeconds, LOW, SaslOAuthBearerClockSkewSecondsDoc) .define(SaslOAuthBearerClockSkewSecondsProp, INT, Defaults.SASL_OAUTH_BEARER_CLOCK_SKEW_SECONDS, LOW, SaslOAuthBearerClockSkewSecondsDoc)
.define(SaslOAuthBearerExpectedAudienceProp, LIST, null, LOW, SaslOAuthBearerExpectedAudienceDoc) .define(SaslOAuthBearerExpectedAudienceProp, LIST, null, LOW, SaslOAuthBearerExpectedAudienceDoc)
.define(SaslOAuthBearerExpectedIssuerProp, STRING, null, LOW, SaslOAuthBearerExpectedIssuerDoc) .define(SaslOAuthBearerExpectedIssuerProp, STRING, null, LOW, SaslOAuthBearerExpectedIssuerDoc)
/** ********* Delegation Token Configuration ****************/ /** ********* Delegation Token Configuration ****************/
.define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc) .define(DelegationTokenSecretKeyAliasProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyAliasDoc)
.define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc) .define(DelegationTokenSecretKeyProp, PASSWORD, null, MEDIUM, DelegationTokenSecretKeyDoc)
.define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DelegationTokenMaxLifeTimeMsDefault, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc) .define(DelegationTokenMaxLifeTimeProp, LONG, Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS, atLeast(1), MEDIUM, DelegationTokenMaxLifeTimeDoc)
.define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DelegationTokenExpiryTimeMsDefault, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc) .define(DelegationTokenExpiryTimeMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS, atLeast(1), MEDIUM, DelegationTokenExpiryTimeMsDoc)
.define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DelegationTokenExpiryCheckIntervalMsDefault, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc) .define(DelegationTokenExpiryCheckIntervalMsProp, LONG, Defaults.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS, atLeast(1), LOW, DelegationTokenExpiryCheckIntervalDoc)
/** ********* Password encryption configuration for dynamic configs *********/ /** ********* Password encryption configuration for dynamic configs *********/
.define(PasswordEncoderSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderSecretDoc) .define(PasswordEncoderSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderSecretDoc)
.define(PasswordEncoderOldSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderOldSecretDoc) .define(PasswordEncoderOldSecretProp, PASSWORD, null, MEDIUM, PasswordEncoderOldSecretDoc)
.define(PasswordEncoderKeyFactoryAlgorithmProp, STRING, null, LOW, PasswordEncoderKeyFactoryAlgorithmDoc) .define(PasswordEncoderKeyFactoryAlgorithmProp, STRING, null, LOW, PasswordEncoderKeyFactoryAlgorithmDoc)
.define(PasswordEncoderCipherAlgorithmProp, STRING, Defaults.PasswordEncoderCipherAlgorithm, LOW, PasswordEncoderCipherAlgorithmDoc) .define(PasswordEncoderCipherAlgorithmProp, STRING, Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, LOW, PasswordEncoderCipherAlgorithmDoc)
.define(PasswordEncoderKeyLengthProp, INT, Defaults.PasswordEncoderKeyLength, atLeast(8), LOW, PasswordEncoderKeyLengthDoc) .define(PasswordEncoderKeyLengthProp, INT, Defaults.PASSWORD_ENCODER_KEY_LENGTH, atLeast(8), LOW, PasswordEncoderKeyLengthDoc)
.define(PasswordEncoderIterationsProp, INT, Defaults.PasswordEncoderIterations, atLeast(1024), LOW, PasswordEncoderIterationsDoc) .define(PasswordEncoderIterationsProp, INT, Defaults.PASSWORD_ENCODER_ITERATIONS, atLeast(1024), LOW, PasswordEncoderIterationsDoc)
/** ********* Raft Quorum Configuration *********/ /** ********* Raft Quorum Configuration *********/
.define(RaftConfig.QUORUM_VOTERS_CONFIG, LIST, Defaults.QuorumVoters, new RaftConfig.ControllerQuorumVotersValidator(), HIGH, RaftConfig.QUORUM_VOTERS_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.QuorumElectionTimeoutMs, null, HIGH, RaftConfig.QUORUM_ELECTION_TIMEOUT_MS_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.QuorumFetchTimeoutMs, null, HIGH, RaftConfig.QUORUM_FETCH_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.QuorumElectionBackoffMs, null, HIGH, RaftConfig.QUORUM_ELECTION_BACKOFF_MAX_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.QuorumLingerMs, null, MEDIUM, RaftConfig.QUORUM_LINGER_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.QuorumRequestTimeoutMs, null, MEDIUM, RaftConfig.QUORUM_REQUEST_TIMEOUT_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.QuorumRetryBackoffMs, null, LOW, RaftConfig.QUORUM_RETRY_BACKOFF_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 **/ /** Internal Configurations **/
// This indicates whether unreleased APIs should be advertised by this node. // This indicates whether unreleased APIs should be advertised by this node.

View File

@ -28,7 +28,7 @@ import java.{time, util}
import kafka.integration.KafkaServerTestHarness import kafka.integration.KafkaServerTestHarness
import kafka.security.authorizer.AclEntry import kafka.security.authorizer.AclEntry
import kafka.server.metadata.KRaftMetadataCache import kafka.server.metadata.KRaftMetadataCache
import kafka.server.{Defaults, DynamicConfig, KafkaConfig} import kafka.server.{DynamicConfig, KafkaConfig}
import kafka.utils.TestUtils._ import kafka.utils.TestUtils._
import kafka.utils.{Log4jController, TestInfoUtils, TestUtils} import kafka.utils.{Log4jController, TestInfoUtils, TestUtils}
import org.apache.kafka.clients.HostResolver 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.utils.{Time, Utils}
import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid} import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicCollection, TopicPartition, TopicPartitionInfo, TopicPartitionReplica, Uuid}
import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT 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.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
@ -1352,7 +1353,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId) newConsumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, testClientId)
// Increase timeouts to avoid having a rebalance during the test // 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.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) val consumer = createConsumer(configOverrides = newConsumerConfig)
try { try {

View File

@ -17,11 +17,12 @@
*/ */
package kafka.network package kafka.network
import kafka.server.{BaseRequestTest, Defaults, KafkaConfig} import kafka.server.{BaseRequestTest, KafkaConfig}
import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol 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.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest 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.ListenersProp, s"$internal://localhost:0, $external://localhost:0")
properties.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$internal:PLAINTEXT, $external:PLAINTEXT") properties.put(KafkaConfig.ListenerSecurityProtocolMapProp, s"$internal:PLAINTEXT, $external:PLAINTEXT")
properties.put(s"listener.name.${internal.toLowerCase}.${KafkaConfig.NumNetworkThreadsProp}", "2") 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 @BeforeEach
@ -51,7 +52,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest {
admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
assertEquals(2, getNumNetworkThreads(internal)) assertEquals(2, getNumNetworkThreads(internal))
TestUtils.createTopicWithAdmin(admin, "test", brokers, controllerServers) TestUtils.createTopicWithAdmin(admin, "test", brokers, controllerServers)
assertEquals(Defaults.NumNetworkThreads, getNumNetworkThreads(external)) assertEquals(Defaults.NUM_NETWORK_THREADS, getNumNetworkThreads(external))
} }
@AfterEach @AfterEach
override def tearDown(): Unit = { override def tearDown(): Unit = {
@ -69,7 +70,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest {
@ValueSource(strings = Array("zk", "kraft")) @ValueSource(strings = Array("zk", "kraft"))
def testDynamicNumNetworkThreads(quorum: String): Unit = { def testDynamicNumNetworkThreads(quorum: String): Unit = {
// Increase the base network thread count // Increase the base network thread count
val newBaseNetworkThreadsCount = Defaults.NumNetworkThreads + 1 val newBaseNetworkThreadsCount = Defaults.NUM_NETWORK_THREADS + 1
var props = new Properties var props = new Properties
props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString) props.put(KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)
reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString)) reconfigureServers(props, (KafkaConfig.NumNetworkThreadsProp, newBaseNetworkThreadsCount.toString))

View File

@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.FileRecords import org.apache.kafka.common.record.FileRecords
import org.apache.kafka.common.utils.Utils 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.server.util.MockTime
import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -51,8 +52,8 @@ object StressTestLog {
scheduler = time.scheduler, scheduler = time.scheduler,
time = time, time = time,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,

View File

@ -29,6 +29,7 @@ import kafka.utils._
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{Time, Utils} 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.{KafkaScheduler, Scheduler}
import org.apache.kafka.server.util.CommandLineUtils import org.apache.kafka.server.util.CommandLineUtils
import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -220,8 +221,8 @@ object TestLinearWriteSpeed {
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
time = Time.SYSTEM, time = Time.SYSTEM,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -18,8 +18,6 @@
package kafka.admin package kafka.admin
import java.util.Properties import java.util.Properties
import kafka.server.Defaults
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.utils.TestInfoUtils import kafka.utils.TestInfoUtils
import org.apache.kafka.clients.consumer.Consumer 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.ByteArrayDeserializer
import org.apache.kafka.common.serialization.ByteArraySerializer import org.apache.kafka.common.serialization.ByteArraySerializer
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.config.Defaults
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -202,7 +201,7 @@ class DeleteOffsetsConsumerGroupCommandIntegrationTest extends ConsumerGroupComm
config.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName) config.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
// Increase timeouts to avoid having a rebalance during the test // 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.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) new KafkaConsumer(config)
} }

View File

@ -17,7 +17,7 @@
package kafka.cluster package kafka.cluster
import kafka.log.LogManager import kafka.log.LogManager
import kafka.server.{Defaults, MetadataCache} import kafka.server.MetadataCache
import kafka.server.checkpoints.OffsetCheckpoints import kafka.server.checkpoints.OffsetCheckpoints
import kafka.server.metadata.MockConfigRepository import kafka.server.metadata.MockConfigRepository
import kafka.utils.TestUtils.{MockAlterPartitionListener, MockAlterPartitionManager} import kafka.utils.TestUtils.{MockAlterPartitionListener, MockAlterPartitionManager}
@ -34,6 +34,7 @@ import org.mockito.Mockito.{mock, when}
import java.io.File import java.io.File
import java.util.Properties import java.util.Properties
import org.apache.kafka.server.common.MetadataVersion 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.server.util.MockTime
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
@ -80,7 +81,7 @@ class AbstractPartitionTest {
alterPartitionManager = TestUtils.createAlterIsrManager() alterPartitionManager = TestUtils.createAlterIsrManager()
alterPartitionListener = TestUtils.createIsrChangeListener() alterPartitionListener = TestUtils.createIsrChangeListener()
partition = new Partition(topicPartition, partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = interBrokerProtocolVersion, interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),

View File

@ -35,6 +35,7 @@ import org.apache.kafka.common.requests.FetchRequest
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.{TopicPartition, Uuid}
import org.apache.kafka.server.common.MetadataVersion 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.server.util.MockTime
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache 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} 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) logManager.startup(Set.empty)
val partition = new Partition(topicPartition, val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = kafka.server.Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => 1L, () => 1L,
@ -299,7 +300,7 @@ class PartitionLockTest extends Logging {
val segments = new LogSegments(log.topicPartition) val segments = new LogSegments(log.topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "")
val maxTransactionTimeout = 5 * 60 * 1000 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( val producerStateManager = new ProducerStateManager(
log.topicPartition, log.topicPartition,
log.dir, log.dir,

View File

@ -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.requests.{AlterPartitionResponse, FetchRequest, ListOffsetsRequest, RequestHeader}
import org.apache.kafka.common.utils.SystemTime import org.apache.kafka.common.utils.SystemTime
import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid} import org.apache.kafka.common.{IsolationLevel, TopicPartition, Uuid}
import org.apache.kafka.server.config.Defaults
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.LeaderRecoveryState
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -416,7 +417,7 @@ class PartitionTest extends AbstractPartitionTest {
partition = new Partition( partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -433,7 +434,7 @@ class PartitionTest extends AbstractPartitionTest {
val segments = new LogSegments(log.topicPartition) val segments = new LogSegments(log.topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "") val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.recordVersion, "")
val maxTransactionTimeoutMs = 5 * 60 * 1000 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( val producerStateManager = new ProducerStateManager(
log.topicPartition, log.topicPartition,
log.dir, log.dir,
@ -1246,7 +1247,7 @@ class PartitionTest extends AbstractPartitionTest {
def testIsUnderMinIsr(): Unit = { def testIsUnderMinIsr(): Unit = {
configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2") configRepository.setTopicConfig(topicPartition.topic, TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
partition = new Partition(topicPartition, partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = interBrokerProtocolVersion, interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1337,7 +1338,7 @@ class PartitionTest extends AbstractPartitionTest {
def testIsReplicaIsrEligibleWithEmptyReplicaMap(): Unit = { def testIsReplicaIsrEligibleWithEmptyReplicaMap(): Unit = {
val mockMetadataCache: KRaftMetadataCache = mock(classOf[KRaftMetadataCache]) val mockMetadataCache: KRaftMetadataCache = mock(classOf[KRaftMetadataCache])
val partition = spy(new Partition(topicPartition, val partition = spy(new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = interBrokerProtocolVersion, interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1569,7 +1570,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1686,7 +1687,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1793,7 +1794,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1885,7 +1886,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -1951,7 +1952,7 @@ class PartitionTest extends AbstractPartitionTest {
addBrokerEpochToMockMetadataCache(metadataCache, replicas) addBrokerEpochToMockMetadataCache(metadataCache, replicas)
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2107,7 +2108,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2190,7 +2191,7 @@ class PartitionTest extends AbstractPartitionTest {
val partition = new Partition( val partition = new Partition(
topicPartition, topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.IBP_3_7_IV2, interBrokerProtocolVersion = MetadataVersion.IBP_3_7_IV2,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2568,7 +2569,7 @@ class PartitionTest extends AbstractPartitionTest {
) )
partition = new Partition(topicPartition, partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = interBrokerProtocolVersion, interBrokerProtocolVersion = interBrokerProtocolVersion,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2688,7 +2689,7 @@ class PartitionTest extends AbstractPartitionTest {
zkIsrManager.start() zkIsrManager.start()
val partition = new Partition(topicPartition, 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 interBrokerProtocolVersion = IBP_2_6_IV0, // shouldn't matter, but set this to a ZK isr version
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2780,7 +2781,7 @@ class PartitionTest extends AbstractPartitionTest {
// Create new Partition object for same topicPartition // Create new Partition object for same topicPartition
val partition2 = new Partition(topicPartition, val partition2 = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2825,7 +2826,7 @@ class PartitionTest extends AbstractPartitionTest {
// Create new Partition object for same topicPartition // Create new Partition object for same topicPartition
val partition2 = new Partition(topicPartition, val partition2 = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -2983,7 +2984,7 @@ class PartitionTest extends AbstractPartitionTest {
cleanerConfig = new CleanerConfig(false), time = time) cleanerConfig = new CleanerConfig(false), time = time)
val spyLogManager = spy(logManager) val spyLogManager = spy(logManager)
val partition = new Partition(topicPartition, val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -3022,7 +3023,7 @@ class PartitionTest extends AbstractPartitionTest {
}).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition))
val partition = new Partition(topicPartition, val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),
@ -3064,7 +3065,7 @@ class PartitionTest extends AbstractPartitionTest {
}).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition)) }).when(spyLogManager).initializingLog(ArgumentMatchers.eq(topicPartition))
val partition = new Partition(topicPartition, val partition = new Partition(topicPartition,
replicaLagTimeMaxMs = Defaults.ReplicaLagTimeMaxMs, replicaLagTimeMaxMs = Defaults.REPLICA_LAG_TIME_MAX_MS,
interBrokerProtocolVersion = MetadataVersion.latestTesting, interBrokerProtocolVersion = MetadataVersion.latestTesting,
localBrokerId = brokerId, localBrokerId = brokerId,
() => defaultBrokerEpoch(brokerId), () => defaultBrokerEpoch(brokerId),

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.record.RecordBatch
import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult} import org.apache.kafka.common.requests.{AddPartitionsToTxnResponse, TransactionResult}
import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch} 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.apache.kafka.server.util.MockScheduler
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -1003,7 +1004,7 @@ class TransactionCoordinatorTest {
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata)))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, txnMetadata))))
val expectedTransition = TxnTransitMetadata(producerId, producerId, (producerEpoch + 1).toShort, RecordBatch.NO_PRODUCER_EPOCH, 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), when(transactionManager.appendTransactionToLog(ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(coordinatorEpoch), ArgumentMatchers.eq(coordinatorEpoch),
@ -1014,7 +1015,7 @@ class TransactionCoordinatorTest {
).thenAnswer(_ => {}) ).thenAnswer(_ => {})
coordinator.startup(() => transactionStatePartitionCount, false) coordinator.startup(() => transactionStatePartitionCount, false)
time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS)
scheduler.tick() scheduler.tick()
verify(transactionManager).timedOutTransactions() verify(transactionManager).timedOutTransactions()
verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId)) verify(transactionManager, times(2)).getTransactionState(ArgumentMatchers.eq(transactionalId))
@ -1063,7 +1064,7 @@ class TransactionCoordinatorTest {
.thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, metadata)))) .thenReturn(Right(Some(CoordinatorEpochAndTxnMetadata(coordinatorEpoch, metadata))))
coordinator.startup(() => transactionStatePartitionCount, false) coordinator.startup(() => transactionStatePartitionCount, false)
time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS)
scheduler.tick() scheduler.tick()
verify(transactionManager).timedOutTransactions() verify(transactionManager).timedOutTransactions()
verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId)) verify(transactionManager).getTransactionState(ArgumentMatchers.eq(transactionalId))
@ -1087,7 +1088,7 @@ class TransactionCoordinatorTest {
val bumpedEpoch = (producerEpoch + 1).toShort val bumpedEpoch = (producerEpoch + 1).toShort
val expectedTransition = TxnTransitMetadata(producerId, producerId, bumpedEpoch, RecordBatch.NO_PRODUCER_EPOCH, txnTimeoutMs, 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), when(transactionManager.appendTransactionToLog(ArgumentMatchers.eq(transactionalId),
ArgumentMatchers.eq(coordinatorEpoch), ArgumentMatchers.eq(coordinatorEpoch),
@ -1098,7 +1099,7 @@ class TransactionCoordinatorTest {
).thenAnswer(_ => capturedErrorsCallback.getValue.apply(Errors.NOT_ENOUGH_REPLICAS)) ).thenAnswer(_ => capturedErrorsCallback.getValue.apply(Errors.NOT_ENOUGH_REPLICAS))
coordinator.startup(() => transactionStatePartitionCount, false) coordinator.startup(() => transactionStatePartitionCount, false)
time.sleep(TransactionStateManager.DefaultAbortTimedOutTransactionsIntervalMs) time.sleep(TransactionStateManagerConfig.DEFAULT_ABORT_TIMED_OUT_TRANSACTIONS_INTERVAL_MS)
scheduler.tick() scheduler.tick()
verify(transactionManager).timedOutTransactions() verify(transactionManager).timedOutTransactions()

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch}
import org.apache.kafka.common.utils.Utils 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.server.util.MockTime
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
import org.junit.jupiter.api.{AfterEach, Tag} import org.junit.jupiter.api.{AfterEach, Tag}
@ -112,8 +113,8 @@ abstract class AbstractLogCleanerIntegrationTest {
time = time, time = time,
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true) keepPartitionMetadataFile = true)

View File

@ -22,6 +22,7 @@ import kafka.utils._
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord} import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
import org.apache.kafka.common.utils.Utils 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.record.BrokerCompressionType
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -63,8 +64,8 @@ class BrokerCompressionTest {
time = time, time = time,
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils 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.server.util.MockTime
import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, LogDirFailureChannel, LogSegment, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, LogDirFailureChannel, LogSegment, LogSegments, LogStartOffsetIncrementReason, ProducerStateManager, ProducerStateManagerConfig}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -53,7 +54,7 @@ class LogCleanerManagerTest extends Logging {
val logConfig: LogConfig = new LogConfig(logProps) val logConfig: LogConfig = new LogConfig(logProps)
val time = new MockTime(1400000000000L, 1000L) // Tue May 13 16:53:20 UTC 2014 for `currentTimeMs` val time = new MockTime(1400000000000L, 1000L) // Tue May 13 16:53:20 UTC 2014 for `currentTimeMs`
val offset = 999 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]() val cleanerCheckpoints: mutable.Map[TopicPartition, Long] = mutable.Map[TopicPartition, Long]()
@ -104,7 +105,7 @@ class LogCleanerManagerTest extends Logging {
val logDirFailureChannel = new LogDirFailureChannel(10) val logDirFailureChannel = new LogDirFailureChannel(10)
val config = createLowRetentionLogConfig(logSegmentSize, TopicConfig.CLEANUP_POLICY_COMPACT) val config = createLowRetentionLogConfig(logSegmentSize, TopicConfig.CLEANUP_POLICY_COMPACT)
val maxTransactionTimeoutMs = 5 * 60 * 1000 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 segments = new LogSegments(tp)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.recordVersion, "") val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(tpDir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxTransactionTimeoutMs, producerStateManagerConfig, time) val producerStateManager = new ProducerStateManager(topicPartition, tpDir, maxTransactionTimeoutMs, producerStateManagerConfig, time)
@ -813,7 +814,7 @@ class LogCleanerManagerTest extends Logging {
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = producerStateManagerConfig, producerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true) keepPartitionMetadataFile = true)
@ -867,7 +868,7 @@ class LogCleanerManagerTest extends Logging {
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = producerStateManagerConfig, producerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.CorruptRecordException import org.apache.kafka.common.errors.CorruptRecordException
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils 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.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.MockTime 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} 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 throttler = new Throttler(desiredRatePerSec = Double.MaxValue, checkIntervalMs = Long.MaxValue, time = time)
val tombstoneRetentionMs = 86400000 val tombstoneRetentionMs = 86400000
val largeTimestamp = Long.MaxValue - tombstoneRetentionMs - 1 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 @AfterEach
def teardown(): Unit = { def teardown(): Unit = {
@ -163,7 +164,7 @@ class LogCleanerTest extends Logging {
val topicPartition = UnifiedLog.parseTopicPartitionName(dir) val topicPartition = UnifiedLog.parseTopicPartitionName(dir)
val logDirFailureChannel = new LogDirFailureChannel(10) val logDirFailureChannel = new LogDirFailureChannel(10)
val maxTransactionTimeoutMs = 5 * 60 * 1000 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 logSegments = new LogSegments(topicPartition)
val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.recordVersion, "") val leaderEpochCache = UnifiedLog.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.recordVersion, "")
val producerStateManager = new ProducerStateManager(topicPartition, dir, val producerStateManager = new ProducerStateManager(topicPartition, dir,
@ -2031,7 +2032,7 @@ class LogCleanerTest extends Logging {
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = producerStateManagerConfig, producerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -24,6 +24,7 @@ import kafka.utils.TestUtils
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record.SimpleRecord import org.apache.kafka.common.record.SimpleRecord
import org.apache.kafka.common.utils.{Time, Utils} 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.server.util.KafkaScheduler
import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -151,8 +152,8 @@ class LogConcurrencyTest {
brokerTopicStats = brokerTopicStats, brokerTopicStats = brokerTopicStats,
time = Time.SYSTEM, time = Time.SYSTEM,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -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.common.utils.{Time, Utils}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.IBP_0_11_0_IV0 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.server.util.{MockTime, Scheduler}
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache 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} 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 = _ var config: KafkaConfig = _
val brokerTopicStats = new BrokerTopicStats val brokerTopicStats = new BrokerTopicStats
val maxTransactionTimeoutMs: Int = 5 * 60 * 1000 val maxTransactionTimeoutMs: Int = 5 * 60 * 1000
val producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false) val producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false)
val producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs val producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS
val tmpDir = TestUtils.tempDir() val tmpDir = TestUtils.tempDir()
val logDir = TestUtils.randomPartitionLogDir(tmpDir) val logDir = TestUtils.randomPartitionLogDir(tmpDir)
var logsToClose: Seq[UnifiedLog] = Seq() var logsToClose: Seq[UnifiedLog] = Seq()
@ -98,7 +99,7 @@ class LogLoaderTest {
val logDirFailureChannel = new LogDirFailureChannel(logDirs.size) val logDirFailureChannel = new LogDirFailureChannel(logDirs.size)
val maxTransactionTimeoutMs = 5 * 60 * 1000 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 // Create a LogManager with some overridden methods to facilitate interception of clean shutdown
// flag and to inject an error // flag and to inject an error
@ -345,7 +346,7 @@ class LogLoaderTest {
def createLogWithInterceptedReads(recoveryPoint: Long): UnifiedLog = { def createLogWithInterceptedReads(recoveryPoint: Long): UnifiedLog = {
val maxTransactionTimeoutMs = 5 * 60 * 1000 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 topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logDirFailureChannel = new LogDirFailureChannel(10) val logDirFailureChannel = new LogDirFailureChannel(10)
// Intercept all segment read calls // Intercept all segment read calls
@ -506,7 +507,7 @@ class LogLoaderTest {
firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch) firstAppendTimestamp, coordinatorEpoch = coordinatorEpoch)
assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp) assertEquals(firstAppendTimestamp, log.producerStateManager.lastEntry(producerId).get.lastTimestamp)
val maxProducerIdExpirationMs = kafka.server.Defaults.ProducerIdExpirationMs val maxProducerIdExpirationMs = Defaults.PRODUCER_ID_EXPIRATION_MS
mockTime.sleep(maxProducerIdExpirationMs) mockTime.sleep(maxProducerIdExpirationMs)
assertEquals(Optional.empty(), log.producerStateManager.lastEntry(producerId)) assertEquals(Optional.empty(), log.producerStateManager.lastEntry(producerId))

View File

@ -28,6 +28,7 @@ import org.apache.kafka.common.errors.OffsetOutOfRangeException
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.{DirectoryId, KafkaException, TopicPartition, Uuid}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} 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.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.mockito.ArgumentMatchers.any import org.mockito.ArgumentMatchers.any
@ -793,7 +794,7 @@ class LogManagerTest {
val segmentBytes = 1024 val segmentBytes = 1024
val log = LogTestUtils.createLog(tpFile, logConfig, brokerTopicStats, time.scheduler, time, 0, 0, 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) assertTrue(expectedSegmentsPerLog > 0)
// calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024 // calculate numMessages to append to logs. It'll create "expectedSegmentsPerLog" log segments with segment.bytes=1024
@ -929,7 +930,7 @@ class LogManagerTest {
recoveryPoint = 0, recoveryPoint = 0,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(5 * 60 * 1000, false), producerStateManagerConfig = new ProducerStateManagerConfig(5 * 60 * 1000, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
scheduler = mockTime.scheduler, scheduler = mockTime.scheduler,
time = mockTime, time = mockTime,
brokerTopicStats = mockBrokerTopicStats, brokerTopicStats = mockBrokerTopicStats,

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{MockTime, Time, Utils} 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.checkpoint.LeaderEpochCheckpoint
import org.apache.kafka.storage.internals.epoch.LeaderEpochFileCache 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} import org.apache.kafka.storage.internals.log.{BatchMetadata, EpochEntry, LogConfig, LogFileUtils, LogSegment, LogSegmentOffsetOverflowException, ProducerStateEntry, ProducerStateManager, ProducerStateManagerConfig, RollParams}
@ -607,7 +608,7 @@ class LogSegmentTest {
topicPartition, topicPartition,
logDir, logDir,
5 * 60 * 1000, 5 * 60 * 1000,
new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
new MockTime() new MockTime()
) )
} }

View File

@ -31,6 +31,7 @@ import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
import java.nio.file.Files import java.nio.file.Files
import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap} import java.util.concurrent.{ConcurrentHashMap, ConcurrentMap}
import org.apache.kafka.common.config.TopicConfig 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.server.util.Scheduler
import org.apache.kafka.storage.internals.checkpoint.LeaderEpochCheckpointFile 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} 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, logStartOffset: Long = 0L,
recoveryPoint: Long = 0L, recoveryPoint: Long = 0L,
maxTransactionTimeoutMs: Int = 5 * 60 * 1000, maxTransactionTimeoutMs: Int = 5 * 60 * 1000,
producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig: ProducerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
lastShutdownClean: Boolean = true, lastShutdownClean: Boolean = true,
topicId: Option[Uuid] = None, topicId: Option[Uuid] = None,
keepPartitionMetadataFile: Boolean = true, keepPartitionMetadataFile: Boolean = true,

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.errors._
import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.{MockTime, Utils} 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.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.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
@ -46,7 +47,7 @@ class ProducerStateManagerTest {
private val partition = new TopicPartition("test", 0) private val partition = new TopicPartition("test", 0)
private val producerId = 1L private val producerId = 1L
private val maxTransactionTimeoutMs = 5 * 60 * 1000 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 lateTransactionTimeoutMs = maxTransactionTimeoutMs + ProducerStateManager.LATE_TRANSACTION_BUFFER_MS
private val time = new MockTime private val time = new MockTime

View File

@ -31,6 +31,7 @@ import org.apache.kafka.common.record.MemoryRecords.RecordFilter
import org.apache.kafka.common.record._ import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse} import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse}
import org.apache.kafka.common.utils.{BufferSupplier, Time, Utils} 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.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler} import org.apache.kafka.server.util.{KafkaScheduler, MockTime, Scheduler}
@ -62,7 +63,7 @@ class UnifiedLogTest {
val logDir = TestUtils.randomPartitionLogDir(tmpDir) val logDir = TestUtils.randomPartitionLogDir(tmpDir)
val mockTime = new MockTime() val mockTime = new MockTime()
var logsToClose: Seq[UnifiedLog] = Seq() 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 def metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
@BeforeEach @BeforeEach
@ -4142,7 +4143,7 @@ class UnifiedLogTest {
time: Time = mockTime, time: Time = mockTime,
maxTransactionTimeoutMs: Int = 60 * 60 * 1000, maxTransactionTimeoutMs: Int = 60 * 60 * 1000,
producerStateManagerConfig: ProducerStateManagerConfig = producerStateManagerConfig, producerStateManagerConfig: ProducerStateManagerConfig = producerStateManagerConfig,
producerIdExpirationCheckIntervalMs: Int = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs: Int = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
lastShutdownClean: Boolean = true, lastShutdownClean: Boolean = true,
topicId: Option[Uuid] = None, topicId: Option[Uuid] = None,
keepPartitionMetadataFile: Boolean = true, keepPartitionMetadataFile: Boolean = true,

View File

@ -23,7 +23,7 @@ import java.util.{Base64, Properties}
import kafka.network.RequestChannel.Session import kafka.network.RequestChannel.Session
import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils} import kafka.security.authorizer.{AclAuthorizer, AuthorizerUtils}
import kafka.security.authorizer.AclEntry.WildcardHost 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.utils.TestUtils
import kafka.zk.KafkaZkClient import kafka.zk.KafkaZkClient
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclOperation} 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.security.token.delegation.{DelegationToken, TokenInformation}
import org.apache.kafka.common.utils.{MockTime, SecurityUtils, Time} import org.apache.kafka.common.utils.{MockTime, SecurityUtils, Time}
import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.config.Defaults
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
@ -57,8 +58,8 @@ class DelegationTokenManagerTest extends QuorumTestHarness {
val tokenManagers = Buffer[DelegationTokenManager]() val tokenManagers = Buffer[DelegationTokenManager]()
val secretKey = "secretKey" val secretKey = "secretKey"
val maxLifeTimeMsDefault = Defaults.DelegationTokenMaxLifeTimeMsDefault val maxLifeTimeMsDefault = Defaults.DELEGATION_TOKEN_MAX_LIFE_TIME_MS
val renewTimeMsDefault = Defaults.DelegationTokenExpiryTimeMsDefault val renewTimeMsDefault = Defaults.DELEGATION_TOKEN_EXPIRY_TIME_MS
var tokenCache: DelegationTokenCache = _ var tokenCache: DelegationTokenCache = _
var props: Properties = _ var props: Properties = _

View File

@ -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.metrics.{JmxReporter, Metrics}
import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.server.authorizer._ 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.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.KafkaScheduler import org.apache.kafka.server.util.KafkaScheduler
@ -630,7 +631,7 @@ class DynamicBrokerConfigTest {
val config = KafkaConfig(props) val config = KafkaConfig(props)
config.dynamicConfig.initialize(None, None) config.dynamicConfig.initialize(None, None)
assertEquals(Defaults.MaxConnections, config.maxConnections) assertEquals(Defaults.MAX_CONNECTIONS, config.maxConnections)
assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, config.messageMaxBytes) assertEquals(LogConfig.DEFAULT_MAX_MESSAGE_BYTES, config.messageMaxBytes)
var newProps = new Properties() var newProps = new Properties()
@ -647,7 +648,7 @@ class DynamicBrokerConfigTest {
config.dynamicConfig.updateDefaultConfig(newProps) config.dynamicConfig.updateDefaultConfig(newProps)
// Invalid value should be skipped and reassigned as default value // 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. // Even if One property is invalid, the below should get correctly updated.
assertEquals(1111, config.messageMaxBytes) assertEquals(1111, config.messageMaxBytes)
} }
@ -842,8 +843,8 @@ class TestDynamicThreadPool() extends BrokerReconfigurable {
} }
override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = { override def reconfigure(oldConfig: KafkaConfig, newConfig: KafkaConfig): Unit = {
assertEquals(Defaults.NumIoThreads, oldConfig.numIoThreads) assertEquals(Defaults.NUM_IO_THREADS, oldConfig.numIoThreads)
assertEquals(Defaults.BackgroundThreads, oldConfig.backgroundThreads) assertEquals(Defaults.BACKGROUND_THREADS, oldConfig.backgroundThreads)
assertEquals(10, newConfig.numIoThreads) assertEquals(10, newConfig.numIoThreads)
assertEquals(100, newConfig.backgroundThreads) assertEquals(100, newConfig.backgroundThreads)

View File

@ -99,7 +99,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.common.{Features, MetadataVersion} 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.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.metrics.ClientMetricsTestUtils
import org.apache.kafka.server.util.{FutureUtils, MockTime} import org.apache.kafka.server.util.{FutureUtils, MockTime}
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchParams, FetchPartitionData, LogConfig} 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(1.toShort),
ArgumentMatchers.eq(0), ArgumentMatchers.eq(0),
ArgumentMatchers.eq(TransactionResult.COMMIT), ArgumentMatchers.eq(TransactionResult.COMMIT),
ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS))
)).thenReturn(CompletableFuture.completedFuture[Void](null)) )).thenReturn(CompletableFuture.completedFuture[Void](null))
when(groupCoordinator.completeTransaction( when(groupCoordinator.completeTransaction(
@ -3127,7 +3127,7 @@ class KafkaApisTest extends Logging {
ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(1.toShort),
ArgumentMatchers.eq(0), ArgumentMatchers.eq(0),
ArgumentMatchers.eq(TransactionResult.ABORT), ArgumentMatchers.eq(TransactionResult.ABORT),
ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS))
)).thenReturn(CompletableFuture.completedFuture[Void](null)) )).thenReturn(CompletableFuture.completedFuture[Void](null))
val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] = val entriesPerPartition: ArgumentCaptor[Map[TopicPartition, MemoryRecords]] =
@ -3136,7 +3136,7 @@ class KafkaApisTest extends Logging {
ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit]) ArgumentCaptor.forClass(classOf[Map[TopicPartition, PartitionResponse] => Unit])
when(replicaManager.appendRecords( when(replicaManager.appendRecords(
ArgumentMatchers.eq(Defaults.RequestTimeoutMs.toLong), ArgumentMatchers.eq(Defaults.REQUEST_TIMEOUT_MS.toLong),
ArgumentMatchers.eq(-1), ArgumentMatchers.eq(-1),
ArgumentMatchers.eq(true), ArgumentMatchers.eq(true),
ArgumentMatchers.eq(AppendOrigin.COORDINATOR), ArgumentMatchers.eq(AppendOrigin.COORDINATOR),
@ -3237,7 +3237,7 @@ class KafkaApisTest extends Logging {
ArgumentMatchers.eq(1.toShort), ArgumentMatchers.eq(1.toShort),
ArgumentMatchers.eq(0), ArgumentMatchers.eq(0),
ArgumentMatchers.eq(TransactionResult.COMMIT), ArgumentMatchers.eq(TransactionResult.COMMIT),
ArgumentMatchers.eq(Duration.ofMillis(Defaults.RequestTimeoutMs)) ArgumentMatchers.eq(Duration.ofMillis(Defaults.REQUEST_TIMEOUT_MS))
)).thenReturn(FutureUtils.failedFuture[Void](error.exception())) )).thenReturn(FutureUtils.failedFuture[Void](error.exception()))
kafkaApis = createKafkaApis(overrideProperties = Map( kafkaApis = createKafkaApis(overrideProperties = Map(
KafkaConfig.NewGroupCoordinatorEnableProp -> "true" KafkaConfig.NewGroupCoordinatorEnableProp -> "true"

View File

@ -36,6 +36,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch} import org.apache.kafka.raft.{KafkaRaftClient, OffsetAndEpoch}
import org.apache.kafka.server.common.ApiMessageAndVersion 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.server.util.MockTime
import org.apache.kafka.snapshot.RecordsSnapshotWriter import org.apache.kafka.snapshot.RecordsSnapshotWriter
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -76,8 +77,8 @@ class DumpLogSegmentsTest {
time = time, time = time,
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, false), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, false),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
logDirFailureChannel = new LogDirFailureChannel(10), logDirFailureChannel = new LogDirFailureChannel(10),
topicId = None, topicId = None,
keepPartitionMetadataFile = true keepPartitionMetadataFile = true

View File

@ -19,10 +19,9 @@ package kafka.utils
import javax.crypto.SecretKeyFactory import javax.crypto.SecretKeyFactory
import kafka.server.Defaults
import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.config.types.Password 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.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -32,9 +31,9 @@ class PasswordEncoderTest {
def testEncodeDecode(): Unit = { def testEncodeDecode(): Unit = {
val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"), val encoder = PasswordEncoder.encrypting(new Password("password-encoder-secret"),
None, None,
Defaults.PasswordEncoderCipherAlgorithm, Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM,
Defaults.PasswordEncoderKeyLength, Defaults.PASSWORD_ENCODER_KEY_LENGTH,
Defaults.PasswordEncoderIterations) Defaults.PASSWORD_ENCODER_ITERATIONS)
val password = "test-password" val password = "test-password"
val encoded = encoder.encode(new Password(password)) val encoded = encoder.encode(new Password(password))
val encodedMap = CoreUtils.parseCsvMap(encoded) val encodedMap = CoreUtils.parseCsvMap(encoded)
@ -96,7 +95,7 @@ class PasswordEncoderTest {
keyFactoryAlg, keyFactoryAlg,
cipherAlg, cipherAlg,
keyLength, keyLength,
Defaults.PasswordEncoderIterations) Defaults.PASSWORD_ENCODER_ITERATIONS)
val password = "test-password" val password = "test-password"
val encoded = encoder.encode(new Password(password)) val encoded = encoder.encode(new Password(password))
verifyEncodedPassword(encoder, password, encoded) verifyEncodedPassword(encoder, password, encoded)
@ -107,10 +106,10 @@ class PasswordEncoderTest {
verifyEncodeDecode(keyFactoryAlg = None, "AES/CBC/PKCS5Padding", keyLength = 128) verifyEncodeDecode(keyFactoryAlg = None, "AES/CBC/PKCS5Padding", keyLength = 128)
verifyEncodeDecode(keyFactoryAlg = None, "AES/CFB/PKCS5Padding", keyLength = 128) verifyEncodeDecode(keyFactoryAlg = None, "AES/CFB/PKCS5Padding", keyLength = 128)
verifyEncodeDecode(keyFactoryAlg = None, "AES/OFB/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 = None, "AES/GCM/NoPadding", keyLength = 128)
verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA256"), Defaults.PasswordEncoderCipherAlgorithm, keyLength = 128) verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA256"), Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, keyLength = 128)
verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA512"), Defaults.PasswordEncoderCipherAlgorithm, keyLength = 128) verifyEncodeDecode(keyFactoryAlg = Some("PBKDF2WithHmacSHA512"), Defaults.PASSWORD_ENCODER_CIPHER_ALGORITHM, keyLength = 128)
} }
private def verifyEncodedPassword(encoder: PasswordEncoder, password: String, encoded: String): Unit = { private def verifyEncodedPassword(encoder: PasswordEncoder, password: String, encoded: String): Unit = {

View File

@ -22,6 +22,7 @@ import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
import kafka.log.{LocalLog, LogLoader, UnifiedLog} import kafka.log.{LocalLog, LogLoader, UnifiedLog}
import kafka.server.BrokerTopicStats import kafka.server.BrokerTopicStats
import kafka.utils.TestUtils.retry import kafka.utils.TestUtils.retry
import org.apache.kafka.server.config.Defaults
import org.apache.kafka.server.util.{KafkaScheduler, MockTime} import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, LogSegments, ProducerStateManager, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, LogSegments, ProducerStateManager, ProducerStateManagerConfig}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -133,8 +134,8 @@ class SchedulerTest {
val logConfig = new LogConfig(new Properties()) val logConfig = new LogConfig(new Properties())
val brokerTopicStats = new BrokerTopicStats val brokerTopicStats = new BrokerTopicStats
val maxTransactionTimeoutMs = 5 * 60 * 1000 val maxTransactionTimeoutMs = 5 * 60 * 1000
val maxProducerIdExpirationMs = kafka.server.Defaults.ProducerIdExpirationMs val maxProducerIdExpirationMs = Defaults.PRODUCER_ID_EXPIRATION_MS
val producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs val producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS
val topicPartition = UnifiedLog.parseTopicPartitionName(logDir) val topicPartition = UnifiedLog.parseTopicPartitionName(logDir)
val logDirFailureChannel = new LogDirFailureChannel(10) val logDirFailureChannel = new LogDirFailureChannel(10)
val segments = new LogSegments(topicPartition) val segments = new LogSegments(topicPartition)

View File

@ -74,6 +74,7 @@ import org.apache.kafka.metadata.properties.MetaProperties
import org.apache.kafka.server.ControllerRequestCompletionHandler import org.apache.kafka.server.ControllerRequestCompletionHandler
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} 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.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
@ -1459,8 +1460,8 @@ object TestUtils extends Logging {
flushStartOffsetCheckpointMs = 10000L, flushStartOffsetCheckpointMs = 10000L,
retentionCheckMs = 1000L, retentionCheckMs = 1000L,
maxTransactionTimeoutMs = 5 * 60 * 1000, maxTransactionTimeoutMs = 5 * 60 * 1000,
producerStateManagerConfig = new ProducerStateManagerConfig(kafka.server.Defaults.ProducerIdExpirationMs, transactionVerificationEnabled), producerStateManagerConfig = new ProducerStateManagerConfig(Defaults.PRODUCER_ID_EXPIRATION_MS, transactionVerificationEnabled),
producerIdExpirationCheckIntervalMs = kafka.server.Defaults.ProducerIdExpirationCheckIntervalMs, producerIdExpirationCheckIntervalMs = Defaults.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS,
scheduler = time.scheduler, scheduler = time.scheduler,
time = time, time = time,
brokerTopicStats = new BrokerTopicStats, brokerTopicStats = new BrokerTopicStats,

View File

@ -94,4 +94,4 @@ public class OffsetConfig {
DEFAULT_OFFSETS_TOPIC_COMPRESSION_TYPE, DEFAULT_OFFSET_COMMIT_TIMEOUT_MS, DEFAULT_OFFSETS_TOPIC_COMPRESSION_TYPE, DEFAULT_OFFSET_COMMIT_TIMEOUT_MS,
DEFAULT_OFFSET_COMMIT_REQUIRED_ACKS); DEFAULT_OFFSET_COMMIT_REQUIRED_ACKS);
} }
} }

View File

@ -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<String> 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<String> 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<? extends KafkaPrincipalBuilder> PRINCIPAL_BUILDER = DefaultKafkaPrincipalBuilder.class;
/** ********* Sasl configuration *********/
public static final String SASL_MECHANISM_INTER_BROKER_PROTOCOL = SaslConfigs.DEFAULT_SASL_MECHANISM;
public static final List<String> 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<String> 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<String> 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;
}

View File

@ -97,6 +97,7 @@ include 'clients',
'streams:upgrade-system-tests-35', 'streams:upgrade-system-tests-35',
'tools', 'tools',
'tools:tools-api', 'tools:tools-api',
'transaction-coordinator',
'trogdor' 'trogdor'
project(":storage:api").name = "storage-api" project(":storage:api").name = "storage-api"

View File

@ -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;
}

View File

@ -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";
}