@ -83,7 +83,7 @@ object Defaults {
@@ -83,7 +83,7 @@ object Defaults {
val BrokerHeartbeatIntervalMs = 2000
val BrokerSessionTimeoutMs = 9000
val MetadataSnapshotMaxNewRecordBytes = 20 * 1024 * 1024
val MetadataSnapshotMaxIntervalMs = TimeUnit . HOURS . toMillis ( 1 ) ;
val MetadataSnapshotMaxIntervalMs = TimeUnit . HOURS . toMillis ( 1 )
val MetadataMaxIdleIntervalMs = 500
val MetadataMaxRetentionBytes = 100 * 1024 * 1024
val DeleteTopicEnable = true
@ -224,7 +224,7 @@ object Defaults {
@@ -224,7 +224,7 @@ object Defaults {
val MetricNumSamples = 2
val MetricSampleWindowMs = 30000
val MetricReporterClasses = ""
val MetricRecordingLevel = Sensor . RecordingLevel . INFO . toString ( )
val MetricRecordingLevel = Sensor . RecordingLevel . INFO . toString
val AutoIncludeJmxReporter = true
@ -241,7 +241,7 @@ object Defaults {
@@ -241,7 +241,7 @@ object Defaults {
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 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 * * * * * * * * * * */
@ -754,7 +754,7 @@ object KafkaConfig {
@@ -754,7 +754,7 @@ object KafkaConfig {
"maximum bytes limit is reached."
val MetadataMaxIdleIntervalMsDoc = "This configuration controls how often the active " +
"controller should write no-op records to the metadata partition. If the value is 0, no-op records " +
s" are not appended to the metadata partition. The default value is ${ Defaults . MetadataMaxIdleIntervalMs } " ;
s" are not appended to the metadata partition. The default value is ${ Defaults . MetadataMaxIdleIntervalMs } "
val ControllerListenerNamesDoc = "A comma-separated list of the names of the listeners used by the controller. This is required " +
"if running in KRaft mode. When communicating with the controller quorum, the broker will always use the first listener in this list.\n " +
"Note: The ZooKeeper-based controller should not set this configuration."
@ -886,7 +886,7 @@ object KafkaConfig {
@@ -886,7 +886,7 @@ object KafkaConfig {
val LogCleanerEnableDoc = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size."
val LogCleanerDeleteRetentionMsDoc = "The amount of time to retain tombstone message markers for log compacted topics. This setting also gives a bound " +
"on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise " +
"tombstones messages may be collected before a consumer completes their scan)." ;
"tombstones messages may be collected before a consumer completes their scan)."
val LogCleanerMinCompactionLagMsDoc = "The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted."
val LogCleanerMaxCompactionLagMsDoc = "The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted."
val LogIndexSizeMaxBytesDoc = "The maximum size in bytes of the offset index"
@ -915,13 +915,13 @@ object KafkaConfig {
@@ -915,13 +915,13 @@ object KafkaConfig {
"broker's timestamp and the message timestamp. The message timestamp can be earlier than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." ;
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime."
val LogMessageTimestampAfterMaxMsDoc = "This configuration sets the allowable timestamp difference between the " +
"message timestamp and the broker's timestamp. The message timestamp can be later than or equal to the broker's " +
"timestamp, with the maximum allowable difference determined by the value set in this configuration. " +
"If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " +
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." ;
"this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime."
val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown"
val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server."
@ -939,7 +939,7 @@ object KafkaConfig {
@@ -939,7 +939,7 @@ object KafkaConfig {
"implement the <code>org.apache.kafka.server.policy.CreateTopicPolicy</code> interface."
val AlterConfigPolicyClassNameDoc = "The alter configs policy class that should be used for validation. The class should " +
"implement the <code>org.apache.kafka.server.policy.AlterConfigPolicy</code> interface."
val LogMessageDownConversionEnableDoc = TopicConfig . MESSAGE_DOWNCONVERSION_ENABLE_DOC ;
val LogMessageDownConversionEnableDoc = TopicConfig . MESSAGE_DOWNCONVERSION_ENABLE_DOC
/* * * * * * * * * * * Replication configuration * * * * * * * * * * */
val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels."
@ -1949,7 +1949,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -1949,7 +1949,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// A user - supplied IBP was given
val configuredVersion = MetadataVersion . fromVersionString ( interBrokerProtocolVersionString )
if ( ! configuredVersion . isKRaftSupported ) {
throw new ConfigException ( s" A non-KRaft version ${ interBrokerProtocolVersionString } given for ${ KafkaConfig . InterBrokerProtocolVersionProp } . " +
throw new ConfigException ( s" A non-KRaft version $interBrokerProtocolVersionString given for ${ KafkaConfig . InterBrokerProtocolVersionProp } . " +
s" The minimum version is ${ MetadataVersion . MINIMUM_KRAFT_VERSION } " )
} else {
warn ( s" ${ KafkaConfig . InterBrokerProtocolVersionProp } is deprecated in KRaft mode as of 3.3 and will only " +
@ -1968,7 +1968,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -1968,7 +1968,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val controlledShutdownEnable = getBoolean ( KafkaConfig . ControlledShutdownEnableProp )
/* * * * * * * * * * * Feature configuration * * * * * * * * * * */
def isFeatureVersioningSupported = interBrokerProtocolVersion . isFeatureVersioningSupported ( )
def isFeatureVersioningSupported = interBrokerProtocolVersion . isFeatureVersioningSupported
/* * * * * * * * * * * Group coordinator configuration * * * * * * * * * * */
val groupMinSessionTimeoutMs = getInt ( KafkaConfig . GroupMinSessionTimeoutMsProp )
@ -2038,12 +2038,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -2038,12 +2038,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol . map { case ( listenerName , _ ) => listenerName }
def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol . map { case ( _ , securityProtocol ) => securityProtocol }
def saslMechanismInterBrokerProtocol = getString ( KafkaConfig . SaslMechanismInterBrokerProtocolProp )
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion . isSaslInterBrokerHandshakeRequestEnabled ( )
val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion . isSaslInterBrokerHandshakeRequestEnabled
/* * * * * * * * * * * DelegationToken Configuration * * * * * * * * * * * * * */
val delegationTokenSecretKey = Option ( getPassword ( KafkaConfig . DelegationTokenSecretKeyProp ) )
. getOrElse ( getPassword ( KafkaConfig . DelegationTokenSecretKeyAliasProp ) )
val tokenAuthEnabled = ( delegationTokenSecretKey != null && ! delegationTokenSecretKey . value . isEmpty )
val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey . value . nonEmpty
val delegationTokenMaxLifeMs = getLong ( KafkaConfig . DelegationTokenMaxLifeTimeProp )
val delegationTokenExpiryTimeMs = getLong ( KafkaConfig . DelegationTokenExpiryTimeMsProp )
val delegationTokenExpiryCheckIntervalMs = getLong ( KafkaConfig . DelegationTokenExpiryCheckIntervalMsProp )
@ -2222,7 +2222,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -2222,7 +2222,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// Topic IDs are used with all self - managed quorum clusters and ZK cluster with IBP greater than or equal to 2.8
def usesTopicId : Boolean =
usesSelfManagedQuorum || interBrokerProtocolVersion . isTopicIdsSupported ( )
usesSelfManagedQuorum || interBrokerProtocolVersion . isTopicIdsSupported
val isRemoteLogStorageSystemEnabled : lang . Boolean = getBoolean ( RemoteLogManagerConfig . REMOTE_LOG_STORAGE_SYSTEM_ENABLE_PROP )
@ -2310,7 +2310,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -2310,7 +2310,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
def validateAdvertisedListenersNonEmptyForBroker ( ) : Unit = {
require ( advertisedListenerNames . nonEmpty ,
"There must be at least one advertised listener." + (
if ( processRoles . contains ( BrokerRole ) ) s" Perhaps all listeners appear in ${ ControllerListenerNamesProp } ? " else "" ) )
if ( processRoles . contains ( BrokerRole ) ) s" Perhaps all listeners appear in $ControllerListenerNamesProp ? " else "" ) )
}
if ( processRoles == Set ( BrokerRole ) ) {
// KRaft broker - only
@ -2434,11 +2434,11 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@@ -2434,11 +2434,11 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
s" ${ KafkaConfig . QueuedMaxBytesProp } must be larger or equal to ${ KafkaConfig . SocketRequestMaxBytesProp } " )
if ( maxConnectionsPerIp == 0 )
require ( ! maxConnectionsPerIpOverrides . is Empty, s" ${ KafkaConfig . MaxConnectionsPerIpProp } can be set to zero only if " +
require ( maxConnectionsPerIpOverrides . non Empty, s" ${ KafkaConfig . MaxConnectionsPerIpProp } can be set to zero only if " +
s" ${ KafkaConfig . MaxConnectionsPerIpOverridesProp } property is set. " )
val invalidAddresses = maxConnectionsPerIpOverrides . keys . filterNot ( address => Utils . validHostPattern ( address ) )
if ( ! invalidAddresses . is Empty)
if ( invalidAddresses . non Empty)
throw new IllegalArgumentException ( s" ${ KafkaConfig . MaxConnectionsPerIpOverridesProp } contains invalid addresses : ${ invalidAddresses . mkString ( "," ) } " )
if ( connectionsMaxIdleMs >= 0 )