mirror of https://github.com/apache/kafka.git
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:
parent
cf90382fb9
commit
62ce551826
25
build.gradle
25
build.gradle
|
@ -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"
|
||||||
|
|
||||||
|
|
|
@ -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" />
|
||||||
|
|
|
@ -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))
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 = {
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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 {
|
||||||
|
|
|
@ -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))
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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),
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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),
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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))
|
||||||
|
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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()
|
||||||
)
|
)
|
||||||
}
|
}
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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 = _
|
||||||
|
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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"
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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 = {
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
|
@ -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"
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
|
@ -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";
|
||||||
|
}
|
Loading…
Reference in New Issue