KAFKA-15853 Move KRAFT configs out of KafkaConfig (#15775)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Omnia Ibrahim 2024-04-27 00:02:31 +01:00 committed by GitHub
parent ba6a73776b
commit d88c15fc3e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
35 changed files with 564 additions and 507 deletions

View File

@ -2782,6 +2782,7 @@ project(':jmh-benchmarks') {
} }
implementation project(':server-common') implementation project(':server-common')
implementation project(':server') implementation project(':server')
implementation project(':raft')
implementation project(':clients') implementation project(':clients')
implementation project(':group-coordinator') implementation project(':group-coordinator')
implementation project(':metadata') implementation project(':metadata')

View File

@ -56,6 +56,7 @@
<allow pkg="org.apache.kafka.metadata"/> <allow pkg="org.apache.kafka.metadata"/>
<allow pkg="org.apache.kafka.timeline" /> <allow pkg="org.apache.kafka.timeline" />
<allow pkg="org.apache.kafka.connect" /> <allow pkg="org.apache.kafka.connect" />
<allow class="org.apache.kafka.raft.QuorumConfig"/>
<subpackage name="cache"> <subpackage name="cache">
</subpackage> </subpackage>

View File

@ -16,9 +16,8 @@
*/ */
package kafka.raft package kafka.raft
import kafka.server.KafkaConfig
import org.apache.kafka.common.config.AbstractConfig import org.apache.kafka.common.config.AbstractConfig
import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
final case class MetadataLogConfig( final case class MetadataLogConfig(
logSegmentBytes: Int, logSegmentBytes: Int,
@ -35,15 +34,15 @@ final case class MetadataLogConfig(
object MetadataLogConfig { object MetadataLogConfig {
def apply(config: AbstractConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = { def apply(config: AbstractConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = {
new MetadataLogConfig( new MetadataLogConfig(
config.getInt(KafkaConfig.MetadataLogSegmentBytesProp), config.getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG),
config.getInt(KafkaConfig.MetadataLogSegmentMinBytesProp), config.getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG),
config.getLong(KafkaConfig.MetadataLogSegmentMillisProp), config.getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG),
config.getLong(KafkaConfig.MetadataMaxRetentionBytesProp), config.getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG),
config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp), config.getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG),
maxBatchSizeInBytes, maxBatchSizeInBytes,
maxFetchSizeInBytes, maxFetchSizeInBytes,
ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT, ServerLogConfigs.LOG_DELETE_DELAY_MS_DEFAULT,
config.getInt(KafkaConfig.NodeIdProp) config.getInt(KRaftConfigs.NODE_ID_CONFIG)
) )
} }
} }

View File

@ -22,7 +22,6 @@ import kafka.raft.KafkaMetadataLog.RetentionMsBreach
import kafka.raft.KafkaMetadataLog.RetentionSizeBreach import kafka.raft.KafkaMetadataLog.RetentionSizeBreach
import kafka.raft.KafkaMetadataLog.SnapshotDeletionReason import kafka.raft.KafkaMetadataLog.SnapshotDeletionReason
import kafka.raft.KafkaMetadataLog.UnknownReason import kafka.raft.KafkaMetadataLog.UnknownReason
import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp}
import kafka.server.{BrokerTopicStats, RequestLocal} import kafka.server.{BrokerTopicStats, RequestLocal}
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.config.TopicConfig
@ -31,7 +30,7 @@ import org.apache.kafka.common.record.{MemoryRecords, Records}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid} import org.apache.kafka.common.{KafkaException, TopicPartition, Uuid}
import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch} import org.apache.kafka.raft.{Isolation, KafkaRaftClient, LogAppendInfo, LogFetchInfo, LogOffsetMetadata, OffsetAndEpoch, OffsetMetadata, ReplicatedLog, ValidOffsetAndEpoch}
import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.Scheduler import org.apache.kafka.server.util.Scheduler
import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals import org.apache.kafka.storage.internals
@ -564,7 +563,7 @@ object KafkaMetadataLog extends Logging {
if (config.logSegmentBytes < config.logSegmentMinBytes) { if (config.logSegmentBytes < config.logSegmentMinBytes) {
throw new InvalidConfigurationException( throw new InvalidConfigurationException(
s"Cannot set $MetadataLogSegmentBytesProp below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}" s"Cannot set ${KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG} below ${config.logSegmentMinBytes}: ${config.logSegmentBytes}"
) )
} else if (defaultLogConfig.retentionMs >= 0) { } else if (defaultLogConfig.retentionMs >= 0) {
throw new InvalidConfigurationException( throw new InvalidConfigurationException(
@ -604,7 +603,7 @@ object KafkaMetadataLog extends Logging {
// Print a warning if users have overridden the internal config // Print a warning if users have overridden the internal config
if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) { if (config.logSegmentMinBytes != KafkaRaftClient.MAX_BATCH_SIZE_BYTES) {
metadataLog.error(s"Overriding $MetadataLogSegmentMinBytesProp is only supported for testing. Setting " + metadataLog.error(s"Overriding ${KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG} is only supported for testing. Setting " +
s"this value too low may lead to an inability to write batches of metadata records.") s"this value too low may lead to an inability to write batches of metadata records.")
} }

View File

@ -21,7 +21,6 @@ import java.{lang, util}
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.{Collections, Properties} import java.util.{Collections, Properties}
import kafka.cluster.EndPoint import kafka.cluster.EndPoint
import kafka.server.KafkaConfig.ControllerListenerNamesProp
import kafka.utils.CoreUtils.parseCsvList import kafka.utils.CoreUtils.parseCsvList
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._ import kafka.utils.Implicits._
@ -48,7 +47,7 @@ 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.{Defaults, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, QuotaConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.server.record.BrokerCompressionType
import org.apache.kafka.server.util.Csv import org.apache.kafka.server.util.Csv
@ -109,33 +108,6 @@ object KafkaConfig {
val DeleteTopicEnableProp = "delete.topic.enable" val DeleteTopicEnableProp = "delete.topic.enable"
val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG) val CompressionTypeProp = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.COMPRESSION_TYPE_CONFIG)
/** KRaft mode configs */
val ProcessRolesProp = "process.roles"
val InitialBrokerRegistrationTimeoutMsProp = "initial.broker.registration.timeout.ms"
val BrokerHeartbeatIntervalMsProp = "broker.heartbeat.interval.ms"
val BrokerSessionTimeoutMsProp = "broker.session.timeout.ms"
val NodeIdProp = "node.id"
val MetadataLogDirProp = "metadata.log.dir"
val MetadataSnapshotMaxNewRecordBytesProp = "metadata.log.max.record.bytes.between.snapshots"
val MetadataSnapshotMaxIntervalMsProp = "metadata.log.max.snapshot.interval.ms"
val ControllerListenerNamesProp = "controller.listener.names"
val SaslMechanismControllerProtocolProp = "sasl.mechanism.controller.protocol"
val MetadataLogSegmentMinBytesProp = "metadata.log.segment.min.bytes"
val MetadataLogSegmentBytesProp = "metadata.log.segment.bytes"
val MetadataLogSegmentMillisProp = "metadata.log.segment.ms"
val MetadataMaxRetentionBytesProp = "metadata.max.retention.bytes"
val MetadataMaxRetentionMillisProp = "metadata.max.retention.ms"
val QuorumVotersProp = QuorumConfig.QUORUM_VOTERS_CONFIG
val MetadataMaxIdleIntervalMsProp = "metadata.max.idle.interval.ms"
val ServerMaxStartupTimeMsProp = "server.max.startup.time.ms"
/** ZK to KRaft Migration configs */
val MigrationEnabledProp = "zookeeper.metadata.migration.enable"
val MigrationMetadataMinBatchSizeProp = "zookeeper.metadata.migration.min.batch.size"
/** Enable eligible leader replicas configs */
val ElrEnabledProp = "eligible.leader.replicas.enable"
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
val AuthorizerClassNameProp = "authorizer.class.name" val AuthorizerClassNameProp = "authorizer.class.name"
val EarlyStartListenersProp = "early.start.listeners" val EarlyStartListenersProp = "early.start.listeners"
@ -204,45 +176,6 @@ object KafkaConfig {
"('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " + "('gzip', 'snappy', 'lz4', 'zstd'). It additionally accepts 'uncompressed' which is equivalent to no compression; and " +
"'producer' which means retain the original compression codec set by the producer." "'producer' which means retain the original compression codec set by the producer."
/** KRaft mode configs */
val ProcessRolesDoc = "The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. " +
"This configuration is only applicable for clusters in KRaft (Kafka Raft) mode (instead of ZooKeeper). Leave this config undefined or empty for ZooKeeper clusters."
val InitialBrokerRegistrationTimeoutMsDoc = "When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process."
val BrokerHeartbeatIntervalMsDoc = "The length of time in milliseconds between broker heartbeats. Used when running in KRaft mode."
val BrokerSessionTimeoutMsDoc = "The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in KRaft mode."
val NodeIdDoc = "The node ID associated with the roles this process is playing when <code>process.roles</code> is non-empty. " +
"This is required configuration when running in KRaft mode."
val MetadataLogDirDoc = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
"If it is not set, the metadata log is placed in the first log directory from log.dirs."
val MetadataSnapshotMaxNewRecordBytesDoc = "This is the maximum number of bytes in the log between the latest " +
"snapshot and the high-watermark needed before generating a new snapshot. The default value is " +
s"${Defaults.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 " +
"either the maximum time interval is reached or the maximum bytes limit is reached."
val MetadataSnapshotMaxIntervalMsDoc = "This is the maximum number of milliseconds to wait to generate a snapshot " +
"if there are committed records in the log that are not included in the latest snapshot. A value of zero disables " +
s"time based snapshot generation. The default value is ${Defaults.METADATA_SNAPSHOT_MAX_INTERVAL_MS}. To generate " +
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 " +
"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.METADATA_MAX_IDLE_INTERVAL_MS}"
val ControllerListenerNamesDoc = "A comma-separated list of the names of the listeners used by the controller. This is required " +
"if running in KRaft mode. When communicating with the controller quorum, the broker will always use the first listener in this list.\n " +
"Note: The ZooKeeper-based controller should not set this configuration."
val SaslMechanismControllerProtocolDoc = "SASL mechanism used for communication with controllers. Default is GSSAPI."
val MetadataLogSegmentBytesDoc = "The maximum size of a single metadata log file."
val MetadataLogSegmentMinBytesDoc = "Override the minimum size for a single metadata log file. This should be used for testing only."
val ServerMaxStartupTimeMsDoc = "The maximum number of milliseconds we will wait for the server to come up. " +
"By default there is no limit. This should be used for testing only."
val MetadataLogSegmentMillisDoc = "The maximum time before a new metadata log file is rolled out (in milliseconds)."
val MetadataMaxRetentionBytesDoc = "The maximum combined size of the metadata log and snapshots before deleting old " +
"snapshots and log files. Since at least one snapshot must exist before any logs can be deleted, this is a soft limit."
val MetadataMaxRetentionMillisDoc = "The number of milliseconds to keep a metadata log file or snapshot before " +
"deleting it. Since at least one snapshot must exist before any logs can be deleted, this is a soft limit."
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements <code>${classOf[Authorizer].getName}</code>" + val AuthorizerClassNameDoc = s"The fully qualified name of a class that implements <code>${classOf[Authorizer].getName}</code>" +
" interface, which is used by the broker for authorization." " interface, which is used by the broker for authorization."
@ -345,27 +278,27 @@ object KafkaConfig {
/* /*
* KRaft mode configs. * KRaft mode configs.
*/ */
.define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc) .define(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, LONG, KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC)
.define(MetadataSnapshotMaxIntervalMsProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_INTERVAL_MS, atLeast(0), HIGH, MetadataSnapshotMaxIntervalMsDoc) .define(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, LONG, KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT, atLeast(0), HIGH, KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC)
.define(ProcessRolesProp, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, ProcessRolesDoc) .define(KRaftConfigs.PROCESS_ROLES_CONFIG, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, KRaftConfigs.PROCESS_ROLES_DOC)
.define(NodeIdProp, INT, Defaults.EMPTY_NODE_ID, null, HIGH, NodeIdDoc) .define(KRaftConfigs.NODE_ID_CONFIG, INT, KRaftConfigs.EMPTY_NODE_ID, null, HIGH, KRaftConfigs.NODE_ID_DOC)
.define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc) .define(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, INT, KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT, null, MEDIUM, KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC)
.define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BROKER_HEARTBEAT_INTERVAL_MS, null, MEDIUM, BrokerHeartbeatIntervalMsDoc) .define(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, INT, KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_DEFAULT, null, MEDIUM, KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_DOC)
.define(BrokerSessionTimeoutMsProp, INT, Defaults.BROKER_SESSION_TIMEOUT_MS, null, MEDIUM, BrokerSessionTimeoutMsDoc) .define(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG, INT, KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_DEFAULT, null, MEDIUM, KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_DOC)
.define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc) .define(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, STRING, null, null, HIGH, KRaftConfigs.CONTROLLER_LISTENER_NAMES_DOC)
.define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc) .define(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_DOC)
.define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc) .define(KRaftConfigs.METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, KRaftConfigs.METADATA_LOG_DIR_DOC)
.define(MetadataLogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc) .define(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_DOC)
.defineInternal(MetadataLogSegmentMinBytesProp, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentMinBytesDoc) .defineInternal(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_DOC)
.define(MetadataLogSegmentMillisProp, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, MetadataLogSegmentMillisDoc) .define(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_DOC)
.define(MetadataMaxRetentionBytesProp, LONG, Defaults.METADATA_MAX_RETENTION_BYTES, null, HIGH, MetadataMaxRetentionBytesDoc) .define(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, KRaftConfigs.METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, KRaftConfigs.METADATA_MAX_RETENTION_BYTES_DOC)
.define(MetadataMaxRetentionMillisProp, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, MetadataMaxRetentionMillisDoc) .define(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_DOC)
.define(MetadataMaxIdleIntervalMsProp, INT, Defaults.METADATA_MAX_IDLE_INTERVAL_MS, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc) .define(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, INT, KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT, atLeast(0), LOW, KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_DOC)
.defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc) .defineInternal(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, LONG, KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_DEFAULT, atLeast(0), MEDIUM, KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_DOC)
.define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration") .define(KRaftConfigs.MIGRATION_ENABLED_CONFIG, BOOLEAN, false, HIGH, KRaftConfigs.MIGRATION_ENABLED_DOC)
.define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas") .define(KRaftConfigs.ELR_ENABLED_CONFIG, BOOLEAN, false, HIGH, KRaftConfigs.ELR_ENABLED_DOC)
.defineInternal(MigrationMetadataMinBatchSizeProp, INT, Defaults.MIGRATION_METADATA_MIN_BATCH_SIZE, atLeast(1), .defineInternal(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG, INT, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DEFAULT, atLeast(1),
MEDIUM, "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft") MEDIUM, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DOC)
/************* Authorizer Configuration ***********/ /************* Authorizer Configuration ***********/
.define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc) .define(AuthorizerClassNameProp, STRING, Defaults.AUTHORIZER_CLASS_NAME, new ConfigDef.NonNullValidator(), LOW, AuthorizerClassNameDoc)
@ -730,11 +663,11 @@ object KafkaConfig {
def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = { def populateSynonyms(input: util.Map[_, _]): util.Map[Any, Any] = {
val output = new util.HashMap[Any, Any](input) val output = new util.HashMap[Any, Any](input)
val brokerId = output.get(KafkaConfig.BrokerIdProp) val brokerId = output.get(KafkaConfig.BrokerIdProp)
val nodeId = output.get(KafkaConfig.NodeIdProp) val nodeId = output.get(KRaftConfigs.NODE_ID_CONFIG)
if (brokerId == null && nodeId != null) { if (brokerId == null && nodeId != null) {
output.put(KafkaConfig.BrokerIdProp, nodeId) output.put(KafkaConfig.BrokerIdProp, nodeId)
} else if (brokerId != null && nodeId == null) { } else if (brokerId != null && nodeId == null) {
output.put(KafkaConfig.NodeIdProp, brokerId) output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId)
} }
output output
} }
@ -869,21 +802,21 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp) val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp)
val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp)
var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) var brokerId: Int = getInt(KafkaConfig.BrokerIdProp)
val nodeId: Int = getInt(KafkaConfig.NodeIdProp) val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG)
val initialRegistrationTimeoutMs: Int = getInt(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp) val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG)
val brokerHeartbeatIntervalMs: Int = getInt(KafkaConfig.BrokerHeartbeatIntervalMsProp) val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG)
val brokerSessionTimeoutMs: Int = getInt(KafkaConfig.BrokerSessionTimeoutMsProp) val brokerSessionTimeoutMs: Int = getInt(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG)
def requiresZookeeper: Boolean = processRoles.isEmpty def requiresZookeeper: Boolean = processRoles.isEmpty
def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp) val migrationEnabled: Boolean = getBoolean(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
val migrationMetadataMinBatchSize: Int = getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp) val migrationMetadataMinBatchSize: Int = getInt(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG)
val elrEnabled: Boolean = getBoolean(KafkaConfig.ElrEnabledProp) val elrEnabled: Boolean = getBoolean(KRaftConfigs.ELR_ENABLED_CONFIG)
private def parseProcessRoles(): Set[ProcessRole] = { private def parseProcessRoles(): Set[ProcessRole] = {
val roles = getList(KafkaConfig.ProcessRolesProp).asScala.map { val roles = getList(KRaftConfigs.PROCESS_ROLES_CONFIG).asScala.map {
case "broker" => ProcessRole.BrokerRole case "broker" => ProcessRole.BrokerRole
case "controller" => ProcessRole.ControllerRole case "controller" => ProcessRole.ControllerRole
case role => throw new ConfigException(s"Unknown process role '$role'" + case role => throw new ConfigException(s"Unknown process role '$role'" +
@ -893,7 +826,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val distinctRoles: Set[ProcessRole] = roles.toSet val distinctRoles: Set[ProcessRole] = roles.toSet
if (distinctRoles.size != roles.size) { if (distinctRoles.size != roles.size) {
throw new ConfigException(s"Duplicate role names found in `${KafkaConfig.ProcessRolesProp}`: $roles") throw new ConfigException(s"Duplicate role names found in `${KRaftConfigs.PROCESS_ROLES_CONFIG}`: $roles")
} }
distinctRoles distinctRoles
@ -904,17 +837,17 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} }
def metadataLogDir: String = { def metadataLogDir: String = {
Option(getString(KafkaConfig.MetadataLogDirProp)) match { Option(getString(KRaftConfigs.METADATA_LOG_DIR_CONFIG)) match {
case Some(dir) => dir case Some(dir) => dir
case None => logDirs.head case None => logDirs.head
} }
} }
def metadataLogSegmentBytes = getInt(KafkaConfig.MetadataLogSegmentBytesProp) def metadataLogSegmentBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG)
def metadataLogSegmentMillis = getLong(KafkaConfig.MetadataLogSegmentMillisProp) def metadataLogSegmentMillis = getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG)
def metadataRetentionBytes = getLong(KafkaConfig.MetadataMaxRetentionBytesProp) def metadataRetentionBytes = getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG)
def metadataRetentionMillis = getLong(KafkaConfig.MetadataMaxRetentionMillisProp) def metadataRetentionMillis = getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG)
val serverMaxStartupTimeMs = getLong(KafkaConfig.ServerMaxStartupTimeMsProp) val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG)
def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp)
def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp)
@ -932,10 +865,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} }
/************* Metadata Configuration ***********/ /************* Metadata Configuration ***********/
val metadataSnapshotMaxNewRecordBytes = getLong(KafkaConfig.MetadataSnapshotMaxNewRecordBytesProp) val metadataSnapshotMaxNewRecordBytes = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG)
val metadataSnapshotMaxIntervalMs = getLong(KafkaConfig.MetadataSnapshotMaxIntervalMsProp) val metadataSnapshotMaxIntervalMs = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG)
val metadataMaxIdleIntervalNs: Option[Long] = { val metadataMaxIdleIntervalNs: Option[Long] = {
val value = TimeUnit.NANOSECONDS.convert(getInt(KafkaConfig.MetadataMaxIdleIntervalMsProp).toLong, TimeUnit.MILLISECONDS) val value = TimeUnit.NANOSECONDS.convert(getInt(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG).toLong, TimeUnit.MILLISECONDS)
if (value > 0) Some(value) else None if (value > 0) Some(value) else None
} }
@ -960,7 +893,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName))
throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " + throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " +
s"listener ${listenerName.value()}, but this is not contained in " + s"listener ${listenerName.value()}, but this is not contained in " +
s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KafkaConfig.ControllerListenerNamesProp}") s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}")
listenerName listenerName
}.toSet }.toSet
} }
@ -1291,7 +1224,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap) CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
def controllerListenerNames: Seq[String] = { def controllerListenerNames: Seq[String] = {
val value = Option(getString(KafkaConfig.ControllerListenerNamesProp)).getOrElse("") val value = Option(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)).getOrElse("")
if (value.isEmpty) { if (value.isEmpty) {
Seq.empty Seq.empty
} else { } else {
@ -1302,7 +1235,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
def controllerListeners: Seq[EndPoint] = def controllerListeners: Seq[EndPoint] =
listeners.filter(l => controllerListenerNames.contains(l.listenerName.value())) listeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
def saslMechanismControllerProtocol: String = getString(KafkaConfig.SaslMechanismControllerProtocolProp) def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG)
def controlPlaneListener: Option[EndPoint] = { def controlPlaneListener: Option[EndPoint] = {
controlPlaneListenerName.map { listenerName => controlPlaneListenerName.map { listenerName =>
@ -1406,7 +1339,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@nowarn("cat=deprecation") @nowarn("cat=deprecation")
private def validateValues(): Unit = { private def validateValues(): Unit = {
if (nodeId != brokerId) { if (nodeId != brokerId) {
throw new ConfigException(s"You must set `${KafkaConfig.NodeIdProp}` to the same value as `${KafkaConfig.BrokerIdProp}`.") throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${KafkaConfig.BrokerIdProp}`.")
} }
if (requiresZookeeper) { if (requiresZookeeper) {
if (zkConnect == null) { if (zkConnect == null) {
@ -1420,12 +1353,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} else { } else {
// KRaft-based metadata quorum // KRaft-based metadata quorum
if (nodeId < 0) { if (nodeId < 0) {
throw new ConfigException(s"Missing configuration `${KafkaConfig.NodeIdProp}` which is required " + throw new ConfigException(s"Missing configuration `${KRaftConfigs.NODE_ID_CONFIG}` which is required " +
s"when `process.roles` is defined (i.e. when running in KRaft mode).") s"when `process.roles` is defined (i.e. when running in KRaft mode).")
} }
if (migrationEnabled) { if (migrationEnabled) {
if (zkConnect == null) { if (zkConnect == null) {
throw new ConfigException(s"If using `${KafkaConfig.MigrationEnabledProp}` in KRaft mode, `${ZkConfigs.ZK_CONNECT_CONFIG}` must also be set.") throw new ConfigException(s"If using `${KRaftConfigs.MIGRATION_ENABLED_CONFIG}` in KRaft mode, `${ZkConfigs.ZK_CONNECT_CONFIG}` must also be set.")
} }
} }
} }
@ -1449,12 +1382,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val voterAddressSpecsByNodeId = QuorumConfig.parseVoterConnections(quorumVoters) val voterAddressSpecsByNodeId = QuorumConfig.parseVoterConnections(quorumVoters)
def validateNonEmptyQuorumVotersForKRaft(): Unit = { def validateNonEmptyQuorumVotersForKRaft(): Unit = {
if (voterAddressSpecsByNodeId.isEmpty) { if (voterAddressSpecsByNodeId.isEmpty) {
throw new ConfigException(s"If using ${KafkaConfig.ProcessRolesProp}, ${KafkaConfig.QuorumVotersProp} must contain a parseable set of voters.") throw new ConfigException(s"If using ${KRaftConfigs.PROCESS_ROLES_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.")
} }
} }
def validateNonEmptyQuorumVotersForMigration(): Unit = { def validateNonEmptyQuorumVotersForMigration(): Unit = {
if (voterAddressSpecsByNodeId.isEmpty) { if (voterAddressSpecsByNodeId.isEmpty) {
throw new ConfigException(s"If using ${KafkaConfig.MigrationEnabledProp}, ${KafkaConfig.QuorumVotersProp} must contain a parseable set of voters.") throw new ConfigException(s"If using ${KRaftConfigs.MIGRATION_ENABLED_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.")
} }
} }
def validateControlPlaneListenerEmptyForKRaft(): Unit = { def validateControlPlaneListenerEmptyForKRaft(): Unit = {
@ -1463,25 +1396,25 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} }
def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = { def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = {
require(!advertisedListenerNames.exists(aln => controllerListenerNames.contains(aln.value())), require(!advertisedListenerNames.exists(aln => controllerListenerNames.contains(aln.value())),
s"The advertised.listeners config must not contain KRaft controller listeners from ${KafkaConfig.ControllerListenerNamesProp} when ${KafkaConfig.ProcessRolesProp} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.") s"The advertised.listeners config must not contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the broker role because Kafka clients that send requests via advertised listeners do not send requests to KRaft controllers -- they only send requests to KRaft brokers.")
} }
def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = { def validateControllerQuorumVotersMustContainNodeIdForKRaftController(): Unit = {
require(voterAddressSpecsByNodeId.containsKey(nodeId), require(voterAddressSpecsByNodeId.containsKey(nodeId),
s"If ${KafkaConfig.ProcessRolesProp} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}") s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains the 'controller' role, the node id $nodeId must be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}")
} }
def validateControllerListenerExistsForKRaftController(): Unit = { def validateControllerListenerExistsForKRaftController(): Unit = {
require(controllerListeners.nonEmpty, require(controllerListeners.nonEmpty,
s"${KafkaConfig.ControllerListenerNamesProp} must contain at least one value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must contain at least one value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role")
} }
def validateControllerListenerNamesMustAppearInListenersForKRaftController(): Unit = { def validateControllerListenerNamesMustAppearInListenersForKRaftController(): Unit = {
val listenerNameValues = listeners.map(_.listenerName.value).toSet val listenerNameValues = listeners.map(_.listenerName.value).toSet
require(controllerListenerNames.forall(cln => listenerNameValues.contains(cln)), require(controllerListenerNames.forall(cln => listenerNameValues.contains(cln)),
s"${KafkaConfig.ControllerListenerNamesProp} must only contain values appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must only contain values appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role")
} }
def validateAdvertisedListenersNonEmptyForBroker(): Unit = { def validateAdvertisedListenersNonEmptyForBroker(): Unit = {
require(advertisedListenerNames.nonEmpty, require(advertisedListenerNames.nonEmpty,
"There must be at least one advertised listener." + ( "There must be at least one advertised listener." + (
if (processRoles.contains(ProcessRole.BrokerRole)) s" Perhaps all listeners appear in $ControllerListenerNamesProp?" else "")) if (processRoles.contains(ProcessRole.BrokerRole)) s" Perhaps all listeners appear in ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}?" else ""))
} }
if (processRoles == Set(ProcessRole.BrokerRole)) { if (processRoles == Set(ProcessRole.BrokerRole)) {
// KRaft broker-only // KRaft broker-only
@ -1490,24 +1423,24 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker() validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker()
// nodeId must not appear in controller.quorum.voters // nodeId must not appear in controller.quorum.voters
require(!voterAddressSpecsByNodeId.containsKey(nodeId), require(!voterAddressSpecsByNodeId.containsKey(nodeId),
s"If ${KafkaConfig.ProcessRolesProp} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${KafkaConfig.QuorumVotersProp}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}") s"If ${KRaftConfigs.PROCESS_ROLES_CONFIG} contains just the 'broker' role, the node id $nodeId must not be included in the set of voters ${QuorumConfig.QUORUM_VOTERS_CONFIG}=${voterAddressSpecsByNodeId.asScala.keySet.toSet}")
// controller.listener.names must be non-empty... // controller.listener.names must be non-empty...
require(controllerListenerNames.nonEmpty, require(controllerListenerNames.nonEmpty,
s"${KafkaConfig.ControllerListenerNamesProp} must contain at least one value when running KRaft with just the broker role") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must contain at least one value when running KRaft with just the broker role")
// controller.listener.names are forbidden in listeners... // controller.listener.names are forbidden in listeners...
require(controllerListeners.isEmpty, require(controllerListeners.isEmpty,
s"${KafkaConfig.ControllerListenerNamesProp} must not contain a value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running KRaft with just the broker role") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not contain a value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running KRaft with just the broker role")
// controller.listener.names must all appear in listener.security.protocol.map // controller.listener.names must all appear in listener.security.protocol.map
controllerListenerNames.foreach { name => controllerListenerNames.foreach { name =>
val listenerName = ListenerName.normalised(name) val listenerName = ListenerName.normalised(name)
if (!effectiveListenerSecurityProtocolMap.contains(listenerName)) { if (!effectiveListenerSecurityProtocolMap.contains(listenerName)) {
throw new ConfigException(s"Controller listener with name ${listenerName.value} defined in " + throw new ConfigException(s"Controller listener with name ${listenerName.value} defined in " +
s"${KafkaConfig.ControllerListenerNamesProp} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} (an explicit security mapping for each controller listener is required if ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} is non-empty, or if there are security protocols other than PLAINTEXT in use)") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} (an explicit security mapping for each controller listener is required if ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG} is non-empty, or if there are security protocols other than PLAINTEXT in use)")
} }
} }
// warn that only the first controller listener is used if there is more than one // warn that only the first controller listener is used if there is more than one
if (controllerListenerNames.size > 1) { if (controllerListenerNames.size > 1) {
warn(s"${KafkaConfig.ControllerListenerNamesProp} has multiple entries; only the first will be used since ${KafkaConfig.ProcessRolesProp}=broker: ${controllerListenerNames.asJava}") warn(s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} has multiple entries; only the first will be used since ${KRaftConfigs.PROCESS_ROLES_CONFIG}=broker: ${controllerListenerNames.asJava}")
} }
validateAdvertisedListenersNonEmptyForBroker() validateAdvertisedListenersNonEmptyForBroker()
} else if (processRoles == Set(ProcessRole.ControllerRole)) { } else if (processRoles == Set(ProcessRole.ControllerRole)) {
@ -1517,12 +1450,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// advertised listeners must be empty when only the controller is configured // advertised listeners must be empty when only the controller is configured
require( require(
getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) == null, getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) == null,
s"The ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} config must be empty when ${KafkaConfig.ProcessRolesProp}=controller" s"The ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} config must be empty when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller"
) )
// listeners should only contain listeners also enumerated in the controller listener // listeners should only contain listeners also enumerated in the controller listener
require( require(
effectiveAdvertisedListeners.isEmpty, effectiveAdvertisedListeners.isEmpty,
s"The ${SocketServerConfigs.LISTENERS_CONFIG} config must only contain KRaft controller listeners from ${KafkaConfig.ControllerListenerNamesProp} when ${KafkaConfig.ProcessRolesProp}=controller" s"The ${SocketServerConfigs.LISTENERS_CONFIG} config must only contain KRaft controller listeners from ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller"
) )
validateControllerQuorumVotersMustContainNodeIdForKRaftController() validateControllerQuorumVotersMustContainNodeIdForKRaftController()
validateControllerListenerExistsForKRaftController() validateControllerListenerExistsForKRaftController()
@ -1541,7 +1474,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
if (migrationEnabled) { if (migrationEnabled) {
validateNonEmptyQuorumVotersForMigration() validateNonEmptyQuorumVotersForMigration()
require(controllerListenerNames.nonEmpty, require(controllerListenerNames.nonEmpty,
s"${KafkaConfig.ControllerListenerNamesProp} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}")
require(interBrokerProtocolVersion.isMigrationSupported, s"Cannot enable ZooKeeper migration without setting " + require(interBrokerProtocolVersion.isMigrationSupported, s"Cannot enable ZooKeeper migration without setting " +
s"'${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG}' to 3.4 or higher") s"'${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG}' to 3.4 or higher")
if (logDirs.size > 1) { if (logDirs.size > 1) {
@ -1552,7 +1485,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} else { } else {
// controller listener names must be empty when not in KRaft mode // controller listener names must be empty when not in KRaft mode
require(controllerListenerNames.isEmpty, require(controllerListenerNames.isEmpty,
s"${KafkaConfig.ControllerListenerNamesProp} must be empty when not running in KRaft mode: ${controllerListenerNames.asJava}") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must be empty when not running in KRaft mode: ${controllerListenerNames.asJava}")
} }
validateAdvertisedListenersNonEmptyForBroker() validateAdvertisedListenersNonEmptyForBroker()
} }

View File

@ -122,7 +122,7 @@ object KafkaRaftServer {
val MetadataTopicId = Uuid.METADATA_TOPIC_ID val MetadataTopicId = Uuid.METADATA_TOPIC_ID
/** /**
* Initialize the configured log directories, including both [[KafkaConfig.MetadataLogDirProp]] * Initialize the configured log directories, including both [[KRaftConfigs.MetadataLogDirProp]]
* and [[KafkaConfig.LOG_DIR_PROP]]. This method performs basic validation to ensure that all * and [[KafkaConfig.LOG_DIR_PROP]]. This method performs basic validation to ensure that all
* directories are accessible and have been initialized with consistent `meta.properties`. * directories are accessible and have been initialized with consistent `meta.properties`.
* *

View File

@ -39,6 +39,7 @@ import org.apache.kafka.raft.{Batch, BatchReader, LeaderAndEpoch, RaftClient, Qu
import org.apache.kafka.security.CredentialProvider import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.common.{Features, MetadataVersion} import org.apache.kafka.server.common.{Features, MetadataVersion}
import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.common.serialization.RecordSerde
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.server.fault.ProcessTerminatingFaultHandler import org.apache.kafka.server.fault.ProcessTerminatingFaultHandler
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils, ShutdownableThread} import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils, ShutdownableThread}
import org.apache.kafka.snapshot.SnapshotReader import org.apache.kafka.snapshot.SnapshotReader
@ -447,7 +448,7 @@ object TestRaftServer extends Logging {
// KafkaConfig requires either `process.roles` or `zookeeper.connect`. Neither are // KafkaConfig requires either `process.roles` or `zookeeper.connect`. Neither are
// actually used by the test server, so we fill in `process.roles` with an arbitrary value. // actually used by the test server, so we fill in `process.roles` with an arbitrary value.
serverProps.put(KafkaConfig.ProcessRolesProp, "controller") serverProps.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
val config = KafkaConfig.fromProps(serverProps, doLog = false) val config = KafkaConfig.fromProps(serverProps, doLog = false)
val throughput = opts.options.valueOf(opts.throughputOpt) val throughput = opts.options.valueOf(opts.throughputOpt)

View File

@ -58,10 +58,13 @@ import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.MetadataProvenance; import org.apache.kafka.image.MetadataProvenance;
import org.apache.kafka.metadata.LeaderRecoveryState; import org.apache.kafka.metadata.LeaderRecoveryState;
import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.server.authorizer.Action; import org.apache.kafka.server.authorizer.Action;
import org.apache.kafka.server.authorizer.AuthorizationResult; import org.apache.kafka.server.authorizer.AuthorizationResult;
import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.config.KRaftConfigs;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.net.InetAddress; import java.net.InetAddress;
@ -550,11 +553,11 @@ class DescribeTopicPartitionsRequestHandlerTest {
1, 1,
(short) 1, (short) 1,
false); false);
properties.put(KafkaConfig.NodeIdProp(), Integer.toString(brokerId)); properties.put(KRaftConfigs.NODE_ID_CONFIG, Integer.toString(brokerId));
properties.put(KafkaConfig.ProcessRolesProp(), "broker"); properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker");
int voterId = brokerId + 1; int voterId = brokerId + 1;
properties.put(KafkaConfig.QuorumVotersProp(), voterId + "@localhost:9093"); properties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, voterId + "@localhost:9093");
properties.put(KafkaConfig.ControllerListenerNamesProp(), "SSL"); properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL");
TestUtils.setIbpAndMessageFormatVersions(properties, MetadataVersion.latestProduction()); TestUtils.setIbpAndMessageFormatVersions(properties, MetadataVersion.latestProduction());
return new KafkaConfig(properties); return new KafkaConfig(properties);
} }

View File

@ -40,6 +40,7 @@ import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble;
import org.apache.kafka.network.SocketServerConfigs; import org.apache.kafka.network.SocketServerConfigs;
import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.server.common.ApiMessageAndVersion; import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.config.KRaftConfigs;
import org.apache.kafka.server.fault.FaultHandler; import org.apache.kafka.server.fault.FaultHandler;
import org.apache.kafka.server.fault.MockFaultHandler; import org.apache.kafka.server.fault.MockFaultHandler;
import org.apache.kafka.storage.internals.log.CleanerConfig; import org.apache.kafka.storage.internals.log.CleanerConfig;
@ -158,17 +159,17 @@ public class KafkaClusterTestKit implements AutoCloseable {
ControllerNode controllerNode = nodes.controllerNodes().get(node.id()); ControllerNode controllerNode = nodes.controllerNodes().get(node.id());
Map<String, String> props = new HashMap<>(configProps); Map<String, String> props = new HashMap<>(configProps);
props.put(KafkaConfig$.MODULE$.ServerMaxStartupTimeMsProp(), props.put(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG,
Long.toString(TimeUnit.MINUTES.toMillis(10))); Long.toString(TimeUnit.MINUTES.toMillis(10)));
props.put(KafkaConfig$.MODULE$.ProcessRolesProp(), roles(node.id())); props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, roles(node.id()));
props.put(KafkaConfig$.MODULE$.NodeIdProp(), props.put(KRaftConfigs.NODE_ID_CONFIG,
Integer.toString(node.id())); Integer.toString(node.id()));
// In combined mode, always prefer the metadata log directory of the controller node. // In combined mode, always prefer the metadata log directory of the controller node.
if (controllerNode != null) { if (controllerNode != null) {
props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(), props.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG,
controllerNode.metadataDirectory()); controllerNode.metadataDirectory());
} else { } else {
props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(), props.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG,
node.metadataDirectory()); node.metadataDirectory());
} }
if (brokerNode != null) { if (brokerNode != null) {
@ -185,7 +186,7 @@ public class KafkaClusterTestKit implements AutoCloseable {
props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners(node.id())); props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners(node.id()));
props.put(INTER_BROKER_LISTENER_NAME_CONFIG, props.put(INTER_BROKER_LISTENER_NAME_CONFIG,
nodes.interBrokerListenerName().value()); nodes.interBrokerListenerName().value());
props.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(), props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG,
"CONTROLLER"); "CONTROLLER");
// Note: we can't accurately set controller.quorum.voters yet, since we don't // Note: we can't accurately set controller.quorum.voters yet, since we don't
// yet know what ports each controller will pick. Set it to a dummy string // yet know what ports each controller will pick. Set it to a dummy string

View File

@ -31,7 +31,7 @@ import org.apache.kafka.common.network.{ListenerName, Mode}
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import scala.collection.mutable import scala.collection.mutable
@ -68,14 +68,14 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
configureListeners(cfgs) configureListeners(cfgs)
modifyConfigs(cfgs) modifyConfigs(cfgs)
if (isZkMigrationTest()) { if (isZkMigrationTest()) {
cfgs.foreach(_.setProperty(KafkaConfig.MigrationEnabledProp, "true")) cfgs.foreach(_.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true"))
} }
if (isNewGroupCoordinatorEnabled()) { if (isNewGroupCoordinatorEnabled()) {
cfgs.foreach(_.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true")) cfgs.foreach(_.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"))
} }
if(isKRaftTest()) { if(isKRaftTest()) {
cfgs.foreach(_.setProperty(KafkaConfig.MetadataLogDirProp, TestUtils.tempDir().getAbsolutePath)) cfgs.foreach(_.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, TestUtils.tempDir().getAbsolutePath))
} }
insertControllerListenersIfNeeded(cfgs) insertControllerListenersIfNeeded(cfgs)
@ -106,7 +106,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
props.foreach { config => props.foreach { config =>
// Add a security protocol for the controller endpoints, if one is not already set. // Add a security protocol for the controller endpoints, if one is not already set.
val securityPairs = config.getProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "").split(",") val securityPairs = config.getProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "").split(",")
val toAdd = config.getProperty(KafkaConfig.ControllerListenerNamesProp, "").split(",").filter( val toAdd = config.getProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "").split(",").filter(
e => !securityPairs.exists(_.startsWith(s"$e:"))) e => !securityPairs.exists(_.startsWith(s"$e:")))
if (toAdd.nonEmpty) { if (toAdd.nonEmpty) {
config.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, (securityPairs ++ config.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, (securityPairs ++

View File

@ -46,6 +46,7 @@ import org.apache.kafka.metadata.bootstrap.BootstrapMetadata
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.quota import org.apache.kafka.server.quota
import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaType} import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaType}
@ -1248,8 +1249,8 @@ class KRaftClusterTest {
new TestKitNodes.Builder(). new TestKitNodes.Builder().
setNumBrokerNodes(3). setNumBrokerNodes(3).
setNumControllerNodes(1).build()). setNumControllerNodes(1).build()).
setConfigProp(KafkaConfig.BrokerHeartbeatIntervalMsProp, 10.toString). setConfigProp(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, 10.toString).
setConfigProp(KafkaConfig.BrokerSessionTimeoutMsProp, 1000.toString). setConfigProp(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG, 1000.toString).
build() build()
try { try {
cluster.format() cluster.format()

View File

@ -26,7 +26,7 @@ import org.apache.kafka.common.Uuid
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs}
import org.junit.jupiter.api.Assertions.{assertThrows, fail} import org.junit.jupiter.api.Assertions.{assertThrows, fail}
import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.extension.ExtendWith
import org.junit.jupiter.api.{Tag, Timeout} import org.junit.jupiter.api.{Tag, Timeout}
@ -62,7 +62,7 @@ class KafkaServerKRaftRegistrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -71,9 +71,9 @@ class KafkaServerKRaftRegistrationTest {
val readyFuture = kraftCluster.controllers().values().asScala.head.controller.waitForReadyBrokers(3) val readyFuture = kraftCluster.controllers().values().asScala.head.controller.waitForReadyBrokers(3)
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -100,7 +100,7 @@ class KafkaServerKRaftRegistrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -108,9 +108,9 @@ class KafkaServerKRaftRegistrationTest {
kraftCluster.startup() kraftCluster.startup()
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT")
assertThrows(classOf[IllegalArgumentException], () => zkCluster.rollingBrokerRestart()) assertThrows(classOf[IllegalArgumentException], () => zkCluster.rollingBrokerRestart())
} finally { } finally {

View File

@ -36,8 +36,9 @@ import org.apache.kafka.common.metadata.FeatureLevelRecord
import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR} import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble.VerificationFlag.{REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig.{AddressSpec, InetAddressSpec} import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler} import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
import org.apache.zookeeper.client.ZKClientConfig import org.apache.zookeeper.client.ZKClientConfig
import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper} import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper}
@ -89,7 +90,7 @@ class KRaftQuorumImplementation(
val controllerServer: ControllerServer, val controllerServer: ControllerServer,
val faultHandlerFactory: FaultHandlerFactory, val faultHandlerFactory: FaultHandlerFactory,
val metadataDir: File, val metadataDir: File,
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]], val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, QuorumConfig.AddressSpec]],
val clusterId: String, val clusterId: String,
val log: Logging, val log: Logging,
val faultHandler: FaultHandler val faultHandler: FaultHandler
@ -321,13 +322,13 @@ abstract class QuorumTestHarness extends Logging {
} }
val props = propsList.head val props = propsList.head
props.putAll(overridingProps) props.putAll(overridingProps)
props.setProperty(KafkaConfig.ServerMaxStartupTimeMsProp, TimeUnit.MINUTES.toMillis(10).toString) props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString)
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true")
if (props.getProperty(KafkaConfig.NodeIdProp) == null) { if (props.getProperty(KRaftConfigs.NODE_ID_CONFIG) == null) {
props.setProperty(KafkaConfig.NodeIdProp, "1000") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1000")
} }
val nodeId = Integer.parseInt(props.getProperty(KafkaConfig.NodeIdProp)) val nodeId = Integer.parseInt(props.getProperty(KRaftConfigs.NODE_ID_CONFIG))
val metadataDir = TestUtils.tempDir() val metadataDir = TestUtils.tempDir()
val metaProperties = new MetaProperties.Builder(). val metaProperties = new MetaProperties.Builder().
setVersion(MetaPropertiesVersion.V1). setVersion(MetaPropertiesVersion.V1).
@ -347,14 +348,14 @@ abstract class QuorumTestHarness extends Logging {
val bootstrapMetadata = BootstrapMetadata.fromRecords(metadataRecords, "test harness") val bootstrapMetadata = BootstrapMetadata.fromRecords(metadataRecords, "test harness")
props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath) props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir.getAbsolutePath)
val proto = controllerListenerSecurityProtocol.toString val proto = controllerListenerSecurityProtocol.toString
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:0") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0")
val config = new KafkaConfig(props) val config = new KafkaConfig(props)
val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, AddressSpec]] val controllerQuorumVotersFuture = new CompletableFuture[util.Map[Integer, QuorumConfig.AddressSpec]]
val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader(). val metaPropertiesEnsemble = new MetaPropertiesEnsemble.Loader().
addMetadataLogDir(metadataDir.getAbsolutePath). addMetadataLogDir(metadataDir.getAbsolutePath).
load() load()
@ -380,7 +381,7 @@ abstract class QuorumTestHarness extends Logging {
controllerQuorumVotersFuture.completeExceptionally(e) controllerQuorumVotersFuture.completeExceptionally(e)
} else { } else {
controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId, controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId,
new InetAddressSpec(new InetSocketAddress("localhost", port)))) new QuorumConfig.InetAddressSpec(new InetSocketAddress("localhost", port))))
} }
}) })
controllerServer.startup() controllerServer.startup()

View File

@ -21,15 +21,16 @@ import java.util.Collections
import kafka.testkit.KafkaClusterTestKit import kafka.testkit.KafkaClusterTestKit
import kafka.testkit.TestKitNodes import kafka.testkit.TestKitNodes
import kafka.utils.TestUtils import kafka.utils.TestUtils
import kafka.server.KafkaConfig.{MetadataMaxIdleIntervalMsProp, MetadataSnapshotMaxNewRecordBytesProp}
import org.apache.kafka.common.utils.BufferSupplier import org.apache.kafka.common.utils.BufferSupplier
import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.metadata.MetadataRecordSerde
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.snapshot.RecordsSnapshotReader import org.apache.kafka.snapshot.RecordsSnapshotReader
import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.Assertions.assertNotEquals import org.junit.jupiter.api.Assertions.assertNotEquals
import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Assertions.assertTrue
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.api.Timeout import org.junit.jupiter.api.Timeout
import scala.jdk.CollectionConverters._ import scala.jdk.CollectionConverters._
@Timeout(120) @Timeout(120)
@ -48,8 +49,8 @@ class RaftClusterSnapshotTest {
.setNumControllerNodes(numberOfControllers) .setNumControllerNodes(numberOfControllers)
.build() .build()
) )
.setConfigProp(MetadataSnapshotMaxNewRecordBytesProp, "10") .setConfigProp(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG, "10")
.setConfigProp(MetadataMaxIdleIntervalMsProp, "0") .setConfigProp(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG, "0")
.build() .build()
) { cluster => ) { cluster =>
cluster.format() cluster.format()

View File

@ -16,7 +16,7 @@
*/ */
package kafka.zk package kafka.zk
import kafka.server.{KRaftCachedControllerId, KafkaConfig} import kafka.server.KRaftCachedControllerId
import kafka.test.{ClusterConfig, ClusterGenerator, ClusterInstance} import kafka.test.{ClusterConfig, ClusterGenerator, ClusterInstance}
import kafka.test.annotation.{AutoStart, ClusterConfigProperty, ClusterTemplate, ClusterTest, Type} import kafka.test.annotation.{AutoStart, ClusterConfigProperty, ClusterTemplate, ClusterTest, Type}
import kafka.test.junit.ClusterTestExtensions import kafka.test.junit.ClusterTestExtensions
@ -49,7 +49,7 @@ import org.apache.kafka.security.authorizer.AclEntry.{WILDCARD_HOST, WILDCARD_PR
import org.apache.kafka.security.PasswordEncoder import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.ControllerRequestCompletionHandler import org.apache.kafka.server.ControllerRequestCompletionHandler
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, ProducerIdsBlock} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion, ProducerIdsBlock}
import org.apache.kafka.server.config.{ConfigType, ZkConfigs} import org.apache.kafka.server.config.{ConfigType, KRaftConfigs, ZkConfigs}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue, fail} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull, assertTrue, fail}
import org.junit.jupiter.api.{Assumptions, Timeout} import org.junit.jupiter.api.{Assumptions, Timeout}
import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.extension.ExtendWith
@ -177,7 +177,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -187,9 +187,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() // This would throw if authorizers weren't allowed zkCluster.rollingBrokerRestart() // This would throw if authorizers weren't allowed
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -309,7 +309,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -319,9 +319,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
@ -443,7 +443,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -453,9 +453,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -507,7 +507,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -517,9 +517,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() // This would throw if authorizers weren't allowed zkCluster.rollingBrokerRestart() // This would throw if authorizers weren't allowed
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -589,7 +589,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -602,9 +602,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -657,7 +657,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -667,9 +667,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -717,7 +717,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -727,9 +727,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()
@ -792,7 +792,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)). setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0). setNumBrokerNodes(0).
setNumControllerNodes(1).build()) setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true") .setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect) .setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build() .build()
try { try {
@ -802,9 +802,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers // Enable migration configs and restart brokers
log.info("Restart brokers in migration mode") log.info("Restart brokers in migration mode")
zkCluster.config().serverProperties().put(KafkaConfig.MigrationEnabledProp, "true") zkCluster.config().serverProperties().put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig()) zkCluster.config().serverProperties().put(QuorumConfig.QUORUM_VOTERS_CONFIG, kraftCluster.quorumVotersConfig())
zkCluster.config().serverProperties().put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") zkCluster.config().serverProperties().put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") zkCluster.config().serverProperties().put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
zkCluster.rollingBrokerRestart() zkCluster.rollingBrokerRestart()
zkCluster.waitForReadyBrokers() zkCluster.waitForReadyBrokers()

View File

@ -17,7 +17,6 @@
package kafka.raft package kafka.raft
import kafka.log.UnifiedLog import kafka.log.UnifiedLog
import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMillisProp, MetadataLogSegmentMinBytesProp, NodeIdProp, ProcessRolesProp, QuorumVotersProp}
import kafka.server.{KafkaConfig, KafkaRaftServer} import kafka.server.{KafkaConfig, KafkaRaftServer}
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException} import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException}
@ -28,7 +27,7 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.raft._ import org.apache.kafka.raft._
import org.apache.kafka.raft.internals.BatchBuilder import org.apache.kafka.raft.internals.BatchBuilder
import org.apache.kafka.server.common.serialization.RecordSerde import org.apache.kafka.server.common.serialization.RecordSerde
import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.MockTime import org.apache.kafka.server.util.MockTime
import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots} import org.apache.kafka.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason} import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason}
@ -62,19 +61,19 @@ final class KafkaMetadataLogTest {
@Test @Test
def testConfig(): Unit = { def testConfig(): Unit = {
val props = new Properties() val props = new Properties()
props.put(ProcessRolesProp, util.Arrays.asList("broker")) props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, util.Arrays.asList("broker"))
props.put(QuorumVotersProp, "1@localhost:9093") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
props.put(NodeIdProp, Int.box(2)) props.put(KRaftConfigs.NODE_ID_CONFIG, Int.box(2))
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.put(MetadataLogSegmentBytesProp, Int.box(10240)) props.put(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240))
props.put(MetadataLogSegmentMillisProp, Int.box(10 * 1024)) props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024))
assertThrows(classOf[InvalidConfigurationException], () => { assertThrows(classOf[InvalidConfigurationException], () => {
val kafkaConfig = KafkaConfig.fromProps(props) val kafkaConfig = KafkaConfig.fromProps(props)
val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
buildMetadataLog(tempDir, mockTime, metadataConfig) buildMetadataLog(tempDir, mockTime, metadataConfig)
}) })
props.put(MetadataLogSegmentMinBytesProp, Int.box(10240)) props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, Int.box(10240))
val kafkaConfig = KafkaConfig.fromProps(props) val kafkaConfig = KafkaConfig.fromProps(props)
val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES) val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
buildMetadataLog(tempDir, mockTime, metadataConfig) buildMetadataLog(tempDir, mockTime, metadataConfig)

View File

@ -26,7 +26,9 @@ 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.internals.FatalExitError import org.apache.kafka.common.internals.FatalExitError
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ZkConfigs} import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ZkConfigs}
import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.server.config.ReplicationConfigs
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test} import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -86,15 +88,15 @@ class KafkaConfigTest {
def testBrokerRoleNodeIdValidation(): Unit = { def testBrokerRoleNodeIdValidation(): Unit = {
// Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters // Ensure that validation is happening at startup to check that brokers do not use their node.id as a voter in controller.quorum.voters
val propertiesFile = new Properties val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "broker") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
setListenerProps(propertiesFile) setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile, assertBadConfigContainingMessage(propertiesFile,
"If process.roles contains just the 'broker' role, the node id 1 must not be included in the set of voters") "If process.roles contains just the 'broker' role, the node id 1 must not be included in the set of voters")
// Ensure that with a valid config no exception is thrown // Ensure that with a valid config no exception is thrown
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile) KafkaConfig.fromProps(propertiesFile)
} }
@ -102,15 +104,15 @@ class KafkaConfigTest {
def testControllerRoleNodeIdValidation(): Unit = { def testControllerRoleNodeIdValidation(): Unit = {
// Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters // Ensure that validation is happening at startup to check that controllers use their node.id as a voter in controller.quorum.voters
val propertiesFile = new Properties val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
setListenerProps(propertiesFile) setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile, assertBadConfigContainingMessage(propertiesFile,
"If process.roles contains the 'controller' role, the node id 1 must be included in the set of voters") "If process.roles contains the 'controller' role, the node id 1 must be included in the set of voters")
// Ensure that with a valid config no exception is thrown // Ensure that with a valid config no exception is thrown
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile) KafkaConfig.fromProps(propertiesFile)
} }
@ -118,24 +120,24 @@ class KafkaConfigTest {
def testCombinedRoleNodeIdValidation(): Unit = { def testCombinedRoleNodeIdValidation(): Unit = {
// Ensure that validation is happening at startup to check that combined processes use their node.id as a voter in controller.quorum.voters // Ensure that validation is happening at startup to check that combined processes use their node.id as a voter in controller.quorum.voters
val propertiesFile = new Properties val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
setListenerProps(propertiesFile) setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile, assertBadConfigContainingMessage(propertiesFile,
"If process.roles contains the 'controller' role, the node id 1 must be included in the set of voters") "If process.roles contains the 'controller' role, the node id 1 must be included in the set of voters")
// Ensure that with a valid config no exception is thrown // Ensure that with a valid config no exception is thrown
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile) KafkaConfig.fromProps(propertiesFile)
} }
@Test @Test
def testIsKRaftCombinedMode(): Unit = { def testIsKRaftCombinedMode(): Unit = {
val propertiesFile = new Properties val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
setListenerProps(propertiesFile) setListenerProps(propertiesFile)
val config = KafkaConfig.fromProps(propertiesFile) val config = KafkaConfig.fromProps(propertiesFile)
assertTrue(config.isKRaftCombinedMode) assertTrue(config.isKRaftCombinedMode)
@ -145,32 +147,32 @@ class KafkaConfigTest {
def testMustContainQuorumVotersIfUsingProcessRoles(): Unit = { def testMustContainQuorumVotersIfUsingProcessRoles(): Unit = {
// Ensure that validation is happening at startup to check that if process.roles is set controller.quorum.voters is not empty // Ensure that validation is happening at startup to check that if process.roles is set controller.quorum.voters is not empty
val propertiesFile = new Properties val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1") propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "") propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "")
setListenerProps(propertiesFile) setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile, assertBadConfigContainingMessage(propertiesFile,
"If using process.roles, controller.quorum.voters must contain a parseable set of voters.") "If using process.roles, controller.quorum.voters must contain a parseable set of voters.")
// Ensure that if neither process.roles nor controller.quorum.voters is populated, then an exception is thrown if zookeeper.connect is not defined // Ensure that if neither process.roles nor controller.quorum.voters is populated, then an exception is thrown if zookeeper.connect is not defined
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "") propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "")
assertBadConfigContainingMessage(propertiesFile, assertBadConfigContainingMessage(propertiesFile,
"Missing required configuration `zookeeper.connect` which has no default value.") "Missing required configuration `zookeeper.connect` which has no default value.")
// Ensure that no exception is thrown once zookeeper.connect is defined (and we clear controller.listener.names) // Ensure that no exception is thrown once zookeeper.connect is defined (and we clear controller.listener.names)
propertiesFile.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") propertiesFile.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
propertiesFile.setProperty(KafkaConfig.ControllerListenerNamesProp, "") propertiesFile.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "")
KafkaConfig.fromProps(propertiesFile) KafkaConfig.fromProps(propertiesFile)
} }
private def setListenerProps(props: Properties): Unit = { private def setListenerProps(props: Properties): Unit = {
val hasBrokerRole = props.getProperty(KafkaConfig.ProcessRolesProp).contains("broker") val hasBrokerRole = props.getProperty(KRaftConfigs.PROCESS_ROLES_CONFIG).contains("broker")
val hasControllerRole = props.getProperty(KafkaConfig.ProcessRolesProp).contains("controller") val hasControllerRole = props.getProperty(KRaftConfigs.PROCESS_ROLES_CONFIG).contains("controller")
val controllerListener = "SASL_PLAINTEXT://localhost:9092" val controllerListener = "SASL_PLAINTEXT://localhost:9092"
val brokerListener = "PLAINTEXT://localhost:9093" val brokerListener = "PLAINTEXT://localhost:9093"
if (hasBrokerRole || hasControllerRole) { // KRaft if (hasBrokerRole || hasControllerRole) { // KRaft
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SASL_PLAINTEXT") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SASL_PLAINTEXT")
if (hasBrokerRole && hasControllerRole) { if (hasBrokerRole && hasControllerRole) {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"$brokerListener,$controllerListener") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"$brokerListener,$controllerListener")
} else if (hasControllerRole) { } else if (hasControllerRole) {

View File

@ -31,9 +31,8 @@ import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.config.{KRaftConfigs,ReplicationConfigs, ServerLogConfigs, ZkConfigs}
import org.apache.kafka.server.ProcessRole import org.apache.kafka.server.ProcessRole
import org.apache.kafka.server.config.ZkConfigs
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -55,11 +54,11 @@ class RaftManagerTest {
} }
if (migrationEnabled) { if (migrationEnabled) {
metadataDir.foreach { value => metadataDir.foreach { value =>
props.setProperty(KafkaConfig.MetadataLogDirProp, value.toString) props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, value.toString)
} }
props.setProperty(KafkaConfig.MigrationEnabledProp, "true") props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
} }
props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
@ -78,23 +77,23 @@ class RaftManagerTest {
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
} }
metadataDir.foreach { value => metadataDir.foreach { value =>
props.setProperty(KafkaConfig.MetadataLogDirProp, value.toString) props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, value.toString)
} }
props.setProperty(KafkaConfig.ProcessRolesProp, processRoles.mkString(",")) props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, processRoles.mkString(","))
props.setProperty(KafkaConfig.NodeIdProp, nodeId.toString) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
if (processRoles.contains(ProcessRole.BrokerRole)) { if (processRoles.contains(ProcessRole.BrokerRole)) {
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "PLAINTEXT") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "PLAINTEXT")
if (processRoles.contains(ProcessRole.ControllerRole)) { // co-located if (processRoles.contains(ProcessRole.ControllerRole)) { // co-located
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093")
props.setProperty(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
} else { // broker-only } else { // broker-only
val voterId = nodeId + 1 val voterId = nodeId + 1
props.setProperty(KafkaConfig.QuorumVotersProp, s"$voterId@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$voterId@localhost:9093")
} }
} else if (processRoles.contains(ProcessRole.ControllerRole)) { // controller-only } else if (processRoles.contains(ProcessRole.ControllerRole)) { // controller-only
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093")
props.setProperty(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
} }
new KafkaConfig(props) new KafkaConfig(props)

View File

@ -25,7 +25,8 @@ import org.apache.kafka.common.message.{BrokerHeartbeatResponseData, BrokerRegis
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
import org.apache.kafka.metadata.BrokerState import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
@ -37,12 +38,12 @@ class BrokerLifecycleManagerTest {
def configProperties = { def configProperties = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "broker") properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
properties.setProperty(KafkaConfig.NodeIdProp, "1") properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9093") properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"2@localhost:9093")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
properties.setProperty(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp, "300000") properties.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "300000")
properties.setProperty(KafkaConfig.BrokerHeartbeatIntervalMsProp, "100") properties.setProperty(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, "100")
properties properties
} }

View File

@ -52,8 +52,10 @@ import org.apache.kafka.common.{ElectionType, Uuid}
import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT
import org.apache.kafka.controller.{Controller, ControllerRequestContext, ResultOrError} import org.apache.kafka.controller.{Controller, ControllerRequestContext, ResultOrError}
import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher import org.apache.kafka.image.publisher.ControllerRegistrationsPublisher
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer} import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult, Authorizer}
import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock} import org.apache.kafka.server.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock}
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.server.util.FutureUtils import org.apache.kafka.server.util.FutureUtils
import org.apache.kafka.storage.internals.log.CleanerConfig import org.apache.kafka.storage.internals.log.CleanerConfig
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -151,10 +153,10 @@ class ControllerApisTest {
controller: Controller, controller: Controller,
props: Properties = new Properties(), props: Properties = new Properties(),
throttle: Boolean = false): ControllerApis = { throttle: Boolean = false): ControllerApis = {
props.put(KafkaConfig.NodeIdProp, nodeId: java.lang.Integer) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId: java.lang.Integer)
props.put(KafkaConfig.ProcessRolesProp, "controller") props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
props.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9092") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9092")
new ControllerApis( new ControllerApis(
requestChannel, requestChannel,
authorizer, authorizer,

View File

@ -27,9 +27,9 @@ import org.apache.kafka.image.loader.{LogDeltaManifest, SnapshotManifest}
import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance} import org.apache.kafka.image.{MetadataDelta, MetadataImage, MetadataProvenance}
import org.apache.kafka.metadata.{ListenerInfo, RecordTestUtils, VersionRange} import org.apache.kafka.metadata.{ListenerInfo, RecordTestUtils, VersionRange}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.LeaderAndEpoch import org.apache.kafka.raft.{LeaderAndEpoch, QuorumConfig}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.apache.kafka.test.TestUtils import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
@ -48,12 +48,12 @@ class ControllerRegistrationManagerTest {
private def configProperties = { private def configProperties = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller") properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
properties.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:PLAINTEXT") properties.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:PLAINTEXT")
properties.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:8001") properties.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:8001")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
properties.setProperty(KafkaConfig.NodeIdProp, "1") properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"1@localhost:8000,2@localhost:5000,3@localhost:7000") properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"1@localhost:8000,2@localhost:5000,3@localhost:7000")
properties properties
} }

View File

@ -33,14 +33,14 @@ import org.apache.kafka.common.config.types.Password
import org.apache.kafka.common.config.{ConfigException, SslConfigs} 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.raft.QuorumConfig
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.security.PasswordEncoderConfigs
import org.apache.kafka.server.authorizer._ import org.apache.kafka.server.authorizer._
import org.apache.kafka.server.config.{Defaults, KafkaSecurityConfigs, ZkConfigs} import org.apache.kafka.server.config.{Defaults, KRaftConfigs, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs}
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
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig} import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig, ProducerStateManagerConfig}
import org.apache.kafka.test.MockMetricsReporter import org.apache.kafka.test.MockMetricsReporter
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -518,10 +518,10 @@ class DynamicBrokerConfigTest {
enableControlledShutdown = true, enableControlledShutdown = true,
enableDeleteTopic = true, enableDeleteTopic = true,
port) port)
retval.put(KafkaConfig.ProcessRolesProp, "broker,controller") retval.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
retval.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") retval.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
retval.put(SocketServerConfigs.LISTENERS_CONFIG, s"${retval.get(SocketServerConfigs.LISTENERS_CONFIG)},CONTROLLER://localhost:0") retval.put(SocketServerConfigs.LISTENERS_CONFIG, s"${retval.get(SocketServerConfigs.LISTENERS_CONFIG)},CONTROLLER://localhost:0")
retval.put(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0") retval.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId}@localhost:0")
retval retval
} }
@ -562,12 +562,12 @@ class DynamicBrokerConfigTest {
enableDeleteTopic = true, enableDeleteTopic = true,
port port
) )
retval.put(KafkaConfig.ProcessRolesProp, "controller") retval.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
retval.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) retval.remove(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
retval.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") retval.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
retval.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:0") retval.put(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:0")
retval.put(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0") retval.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId}@localhost:0")
retval retval
} }
@ -713,11 +713,11 @@ class DynamicBrokerConfigTest {
def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = { def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = {
val props = new Properties() val props = new Properties()
props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181") props.put(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
props.put(KafkaConfig.MetadataLogSegmentMinBytesProp, "1024") props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG, "1024")
val config = new KafkaConfig(props) val config = new KafkaConfig(props)
assertFalse(config.nonInternalValues.containsKey(KafkaConfig.MetadataLogSegmentMinBytesProp)) assertFalse(config.nonInternalValues.containsKey(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG))
config.updateCurrentConfig(new KafkaConfig(props)) config.updateCurrentConfig(new KafkaConfig(props))
assertFalse(config.nonInternalValues.containsKey(KafkaConfig.MetadataLogSegmentMinBytesProp)) assertFalse(config.nonInternalValues.containsKey(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG))
} }
@Test @Test

View File

@ -74,12 +74,12 @@ import org.apache.kafka.common.utils.{ProducerIdAndEpoch, SecurityUtils, Utils}
import org.apache.kafka.common._ import org.apache.kafka.common._
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.{GroupCoordinator, GroupCoordinatorConfig}
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.ClientMetricsManager import org.apache.kafka.server.ClientMetricsManager
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.config.{ConfigType, Defaults, KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer} import org.apache.kafka.server.authorizer.{Action, AuthorizationResult, Authorizer}
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.common.{Features, MetadataVersion} import org.apache.kafka.server.common.{Features, MetadataVersion}
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}
@ -153,11 +153,11 @@ class KafkaApisTest extends Logging {
overrideProperties: Map[String, String] = Map.empty): KafkaApis = { overrideProperties: Map[String, String] = Map.empty): KafkaApis = {
val properties = if (raftSupport) { val properties = if (raftSupport) {
val properties = TestUtils.createBrokerConfig(brokerId, "") val properties = TestUtils.createBrokerConfig(brokerId, "")
properties.put(KafkaConfig.NodeIdProp, brokerId.toString) properties.put(KRaftConfigs.NODE_ID_CONFIG, brokerId.toString)
properties.put(KafkaConfig.ProcessRolesProp, "broker") properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
val voterId = brokerId + 1 val voterId = brokerId + 1
properties.put(KafkaConfig.QuorumVotersProp, s"$voterId@localhost:9093") properties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$voterId@localhost:9093")
properties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
properties properties
} else { } else {
TestUtils.createBrokerConfig(brokerId, "zk") TestUtils.createBrokerConfig(brokerId, "zk")

View File

@ -26,7 +26,7 @@ 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, Records} import org.apache.kafka.common.record.{CompressionType, Records}
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.raft.QuorumConfig.{AddressSpec, InetAddressSpec, UNKNOWN_ADDRESS_SPEC_INSTANCE} import org.apache.kafka.raft.QuorumConfig
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -40,10 +40,9 @@ import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, Transact
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.PasswordEncoderConfigs import org.apache.kafka.security.PasswordEncoderConfigs
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1} import org.apache.kafka.server.common.MetadataVersion.{IBP_0_8_2, IBP_3_0_IV1}
import org.apache.kafka.server.config.{KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, QuotaConfigs, ServerTopicConfigSynonyms, ZkConfigs} import org.apache.kafka.server.config.{KRaftConfigs, KafkaSecurityConfigs, ReplicationConfigs, ServerLogConfigs, QuotaConfigs, ServerTopicConfigSynonyms, ZkConfigs}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.storage.internals.log.CleanerConfig import org.apache.kafka.storage.internals.log.CleanerConfig
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
@ -286,11 +285,11 @@ class KafkaConfigTest {
@Test @Test
def testControllerListenerNames(): Unit = { def testControllerListenerNames(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:0,CONTROLLER://localhost:5000")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:5000") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:5000")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SASL_SSL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SASL_SSL")
val serverConfig = KafkaConfig.fromProps(props) val serverConfig = KafkaConfig.fromProps(props)
@ -305,11 +304,11 @@ class KafkaConfigTest {
@Test @Test
def testControlPlaneListenerNameNotAllowedWithKRaft(): Unit = { def testControlPlaneListenerNameNotAllowedWithKRaft(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
props.setProperty(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, "SSL") props.setProperty(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, "SSL")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
@ -322,50 +321,50 @@ class KafkaConfigTest {
@Test @Test
def testControllerListenerDefinedForKRaftController(): Unit = { def testControllerListenerDefinedForKRaftController(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertBadConfigContainingMessage(props, "The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller") assertBadConfigContainingMessage(props, "The listeners config must only contain KRaft controller listeners from controller.listener.names when process.roles=controller")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
// confirm that redirecting via listener.security.protocol.map is acceptable // confirm that redirecting via listener.security.protocol.map is acceptable
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:SSL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:SSL")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@Test @Test
def testControllerListenerDefinedForKRaftBroker(): Unit = { def testControllerListenerDefinedForKRaftBroker(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
assertBadConfigContainingMessage(props, "controller.listener.names must contain at least one value when running KRaft with just the broker role") assertBadConfigContainingMessage(props, "controller.listener.names must contain at least one value when running KRaft with just the broker role")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
// confirm that redirecting via listener.security.protocol.map is acceptable // confirm that redirecting via listener.security.protocol.map is acceptable
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@Test @Test
def testPortInQuorumVotersNotRequiredToMatchFirstControllerListenerPortForThisKRaftController(): Unit = { def testPortInQuorumVotersNotRequiredToMatchFirstControllerListenerPortForThisKRaftController(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:9094") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093,3@anotherhost:9094")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL,SASL_SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL,SASL_SSL")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
// change each of the 4 ports to port 5555 -- should pass in all circumstances since we can't validate the // change each of the 4 ports to port 5555 -- should pass in all circumstances since we can't validate the
@ -377,20 +376,20 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:5555") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:5555")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") // reset to original value props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,SSL://localhost:9093,SASL_SSL://localhost:9094") // reset to original value
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:5555,3@anotherhost:9094") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:5555,3@anotherhost:9094")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:5555") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093,3@anotherhost:5555")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@Test @Test
def testSeparateControllerListenerDefinedForKRaftBrokerController(): Unit = { def testSeparateControllerListenerDefinedForKRaftBrokerController(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
assertBadConfigContainingMessage(props, "There must be at least one advertised listener. Perhaps all listeners appear in controller.listener.names?") assertBadConfigContainingMessage(props, "There must be at least one advertised listener. Perhaps all listeners appear in controller.listener.names?")
@ -401,26 +400,26 @@ class KafkaConfigTest {
// confirm that redirecting via listener.security.protocol.map is acceptable // confirm that redirecting via listener.security.protocol.map is acceptable
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,CONTROLLER://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092,CONTROLLER://localhost:9093")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@Test @Test
def testControllerListenerNameMapsToPlaintextByDefaultForKRaft(): Unit = { def testControllerListenerNameMapsToPlaintextByDefaultForKRaft(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
val controllerListenerName = new ListenerName("CONTROLLER") val controllerListenerName = new ListenerName("CONTROLLER")
assertEquals(Some(SecurityProtocol.PLAINTEXT), assertEquals(Some(SecurityProtocol.PLAINTEXT),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName)) KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName))
// ensure we don't map it to PLAINTEXT when there is a SSL or SASL controller listener // ensure we don't map it to PLAINTEXT when there is a SSL or SASL controller listener
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER,SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER,SSL")
val controllerNotFoundInMapMessage = "Controller listener with name CONTROLLER defined in controller.listener.names not found in listener.security.protocol.map" val controllerNotFoundInMapMessage = "Controller listener with name CONTROLLER defined in controller.listener.names not found in listener.security.protocol.map"
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
// ensure we don't map it to PLAINTEXT when there is a SSL or SASL listener // ensure we don't map it to PLAINTEXT when there is a SSL or SASL listener
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9092") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "SSL://localhost:9092")
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
props.remove(SocketServerConfigs.LISTENERS_CONFIG) props.remove(SocketServerConfigs.LISTENERS_CONFIG)
@ -434,7 +433,7 @@ class KafkaConfigTest {
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage) assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
// ensure we can map it to a non-PLAINTEXT security protocol by default (i.e. when nothing is given) // ensure we can map it to a non-PLAINTEXT security protocol by default (i.e. when nothing is given)
props.remove(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG) props.remove(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
assertEquals(Some(SecurityProtocol.SSL), assertEquals(Some(SecurityProtocol.SSL),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("SSL"))) KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("SSL")))
} }
@ -442,11 +441,11 @@ class KafkaConfigTest {
@Test @Test
def testMultipleControllerListenerNamesMapToPlaintextByDefaultForKRaft(): Unit = { def testMultipleControllerListenerNamesMapToPlaintextByDefaultForKRaft(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER1://localhost:9092,CONTROLLER2://localhost:9093") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER1://localhost:9092,CONTROLLER2://localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER1,CONTROLLER2") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER1,CONTROLLER2")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
assertEquals(Some(SecurityProtocol.PLAINTEXT), assertEquals(Some(SecurityProtocol.PLAINTEXT),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER1"))) KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER1")))
assertEquals(Some(SecurityProtocol.PLAINTEXT), assertEquals(Some(SecurityProtocol.PLAINTEXT),
@ -811,18 +810,18 @@ class KafkaConfigTest {
case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
// KRaft mode configs // KRaft mode configs
case KafkaConfig.ProcessRolesProp => // ignore case KRaftConfigs.PROCESS_ROLES_CONFIG => // ignore
case KafkaConfig.InitialBrokerRegistrationTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.BrokerHeartbeatIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.BrokerSessionTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.NodeIdProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.NODE_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataLogDirProp => // ignore string case KRaftConfigs.METADATA_LOG_DIR_CONFIG => // ignore string
case KafkaConfig.MetadataLogSegmentBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataLogSegmentMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataMaxRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataMaxRetentionMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.ControllerListenerNamesProp => // ignore string case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string
case KafkaConfig.MetadataMaxIdleIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number") case KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.AuthorizerClassNameProp => //ignore string case KafkaConfig.AuthorizerClassNameProp => //ignore string
case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string case ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG => //ignore string
@ -945,7 +944,7 @@ class KafkaConfigTest {
case KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string case KafkaSecurityConfigs.SSL_PRINCIPAL_MAPPING_RULES_CONFIG => // ignore string
//Sasl Configs //Sasl Configs
case KafkaConfig.SaslMechanismControllerProtocolProp => // ignore case KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG => // ignore
case KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore case KafkaSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG => // ignore
case KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG => case KafkaSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG =>
case KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG => case KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG =>
@ -1221,12 +1220,12 @@ class KafkaConfigTest {
@Test @Test
def testDistinctControllerAndAdvertisedListenersAllowedForKRaftBroker(): Unit = { def testDistinctControllerAndAdvertisedListenersAllowedForKRaftBroker(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094")
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://A:9092,SSL://B:9093") // explicitly setting it in KRaft props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://A:9092,SSL://B:9093") // explicitly setting it in KRaft
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SASL_SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SASL_SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "3@localhost:9094") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3@localhost:9094")
// invalid due to extra listener also appearing in controller listeners // invalid due to extra listener also appearing in controller listeners
assertBadConfigContainingMessage(props, assertBadConfigContainingMessage(props,
@ -1245,14 +1244,14 @@ class KafkaConfigTest {
@Test @Test
def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = { def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker,controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094"
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners)
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) // explicitly setting it in KRaft props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) // explicitly setting it in KRaft
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "SASL_SSL") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "SASL_SSL")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT,SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT,SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
assertBadConfigContainingMessage(props, assertBadConfigContainingMessage(props,
"The advertised.listeners config must not contain KRaft controller listeners from controller.listener.names when process.roles contains the broker role") "The advertised.listeners config must not contain KRaft controller listeners from controller.listener.names when process.roles contains the broker role")
@ -1276,12 +1275,12 @@ class KafkaConfigTest {
val correctControllerListenerNames = "PLAINTEXT,SSL" val correctControllerListenerNames = "PLAINTEXT,SSL"
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners)
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, incorrectListeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, incorrectListeners)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames) props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller" var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller"
assertBadConfigContainingMessage(props, expectedExceptionContainsText) assertBadConfigContainingMessage(props, expectedExceptionContainsText)
@ -1298,13 +1297,13 @@ class KafkaConfigTest {
// Invalid if listeners doesn't contain every name in controller.listener.names // Invalid if listeners doesn't contain every name in controller.listener.names
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames + ",SASL_SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames + ",SASL_SSL")
expectedExceptionContainsText = """controller.listener.names must only contain values appearing in the 'listeners' expectedExceptionContainsText = """controller.listener.names must only contain values appearing in the 'listeners'
|configuration when running the KRaft controller role""".stripMargin.replaceAll("\n", " ") |configuration when running the KRaft controller role""".stripMargin.replaceAll("\n", " ")
assertBadConfigContainingMessage(props, expectedExceptionContainsText) assertBadConfigContainingMessage(props, expectedExceptionContainsText)
// Valid now // Valid now
props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames) props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@ -1336,32 +1335,32 @@ class KafkaConfigTest {
private def assertInvalidQuorumVoters(value: String): Unit = { private def assertInvalidQuorumVoters(value: String): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect)
props.setProperty(KafkaConfig.QuorumVotersProp, value) props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, value)
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
} }
@Test @Test
def testValidQuorumVotersConfig(): Unit = { def testValidQuorumVotersConfig(): Unit = {
val expected = new util.HashMap[Integer, AddressSpec]() val expected = new util.HashMap[Integer, QuorumConfig.AddressSpec]()
assertValidQuorumVoters("", expected) assertValidQuorumVoters("", expected)
expected.put(1, new InetAddressSpec(new InetSocketAddress("127.0.0.1", 9092))) expected.put(1, new QuorumConfig.InetAddressSpec(new InetSocketAddress("127.0.0.1", 9092)))
assertValidQuorumVoters("1@127.0.0.1:9092", expected) assertValidQuorumVoters("1@127.0.0.1:9092", expected)
expected.clear() expected.clear()
expected.put(1, UNKNOWN_ADDRESS_SPEC_INSTANCE) expected.put(1, QuorumConfig.UNKNOWN_ADDRESS_SPEC_INSTANCE)
assertValidQuorumVoters("1@0.0.0.0:0", expected) assertValidQuorumVoters("1@0.0.0.0:0", expected)
expected.clear() expected.clear()
expected.put(1, new InetAddressSpec(new InetSocketAddress("kafka1", 9092))) expected.put(1, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka1", 9092)))
expected.put(2, new InetAddressSpec(new InetSocketAddress("kafka2", 9092))) expected.put(2, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka2", 9092)))
expected.put(3, new InetAddressSpec(new InetSocketAddress("kafka3", 9092))) expected.put(3, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka3", 9092)))
assertValidQuorumVoters("1@kafka1:9092,2@kafka2:9092,3@kafka3:9092", expected) assertValidQuorumVoters("1@kafka1:9092,2@kafka2:9092,3@kafka3:9092", expected)
} }
private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, AddressSpec]): Unit = { private def assertValidQuorumVoters(value: String, expectedVoters: util.Map[Integer, QuorumConfig.AddressSpec]): Unit = {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect)
props.setProperty(KafkaConfig.QuorumVotersProp, value) props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, value)
val raftConfig = new QuorumConfig(KafkaConfig.fromProps(props)) val raftConfig = new QuorumConfig(KafkaConfig.fromProps(props))
assertEquals(expectedVoters, raftConfig.quorumVoterConnections()) assertEquals(expectedVoters, raftConfig.quorumVoterConnections())
} }
@ -1373,11 +1372,11 @@ class KafkaConfigTest {
// and make sure it is allowed despite broker.id.generation.enable=true (true is the default) // and make sure it is allowed despite broker.id.generation.enable=true (true is the default)
val largeBrokerId = 2000 val largeBrokerId = 2000
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
props.setProperty(KafkaConfig.NodeIdProp, largeBrokerId.toString) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, largeBrokerId.toString)
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@ -1385,7 +1384,7 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedBrokerCaseWithAutoGenEnabled(): Unit = { def testRejectsNegativeNodeIdForRaftBasedBrokerCaseWithAutoGenEnabled(): Unit = {
// -1 is the default for both node.id and broker.id // -1 is the default for both node.id and broker.id
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
} }
@ -1393,7 +1392,7 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedControllerCaseWithAutoGenEnabled(): Unit = { def testRejectsNegativeNodeIdForRaftBasedControllerCaseWithAutoGenEnabled(): Unit = {
// -1 is the default for both node.id and broker.id // -1 is the default for both node.id and broker.id
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
} }
@ -1401,9 +1400,9 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedCaseWithAutoGenDisabled(): Unit = { def testRejectsNegativeNodeIdForRaftBasedCaseWithAutoGenDisabled(): Unit = {
// -1 is the default for both node.id and broker.id // -1 is the default for both node.id and broker.id
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false") props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
} }
@ -1438,7 +1437,7 @@ class KafkaConfigTest {
val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094" val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094"
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners)
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners)
props.setProperty(KafkaConfig.NodeIdProp, negativeTwoNodeId.toString) props.setProperty(KRaftConfigs.NODE_ID_CONFIG, negativeTwoNodeId.toString)
props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString) props.setProperty(KafkaConfig.BrokerIdProp, negativeTwoNodeId.toString)
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
} }
@ -1469,7 +1468,7 @@ class KafkaConfigTest {
@Test @Test
def testZookeeperConnectRequiredIfEmptyProcessRoles(): Unit = { def testZookeeperConnectRequiredIfEmptyProcessRoles(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092")
assertFalse(isValidKafkaConfig(props)) assertFalse(isValidKafkaConfig(props))
} }
@ -1477,11 +1476,11 @@ class KafkaConfigTest {
@Test @Test
def testZookeeperConnectNotRequiredIfNonEmptyProcessRoles(): Unit = { def testZookeeperConnectNotRequiredIfNonEmptyProcessRoles(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@ -1491,12 +1490,12 @@ class KafkaConfigTest {
val dataDir = "/path/to/data/dir" val dataDir = "/path/to/data/dir"
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir) props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir)
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataDir) props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataDir)
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
@ -1510,11 +1509,11 @@ class KafkaConfigTest {
val dataDir2 = "/path/to/data/dir/2" val dataDir2 = "/path/to/data/dir/2"
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, s"$dataDir1,$dataDir2") props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, s"$dataDir1,$dataDir2")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
@ -1533,17 +1532,17 @@ class KafkaConfigTest {
input.put(KafkaConfig.BrokerIdProp, "4") input.put(KafkaConfig.BrokerIdProp, "4")
val expectedOutput = new util.HashMap[String, String]() val expectedOutput = new util.HashMap[String, String]()
expectedOutput.put(KafkaConfig.BrokerIdProp, "4") expectedOutput.put(KafkaConfig.BrokerIdProp, "4")
expectedOutput.put(KafkaConfig.NodeIdProp, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4")
assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input))
} }
@Test @Test
def testPopulateSynonymsOnMapWithoutBrokerId(): Unit = { def testPopulateSynonymsOnMapWithoutBrokerId(): Unit = {
val input = new util.HashMap[String, String]() val input = new util.HashMap[String, String]()
input.put(KafkaConfig.NodeIdProp, "4") input.put(KRaftConfigs.NODE_ID_CONFIG, "4")
val expectedOutput = new util.HashMap[String, String]() val expectedOutput = new util.HashMap[String, String]()
expectedOutput.put(KafkaConfig.BrokerIdProp, "4") expectedOutput.put(KafkaConfig.BrokerIdProp, "4")
expectedOutput.put(KafkaConfig.NodeIdProp, "4") expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4")
assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input)) assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input))
} }
@ -1551,7 +1550,7 @@ class KafkaConfigTest {
def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = { def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.BrokerIdProp, "1") props.setProperty(KafkaConfig.BrokerIdProp, "1")
props.setProperty(KafkaConfig.NodeIdProp, "2") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
assertEquals("You must set `node.id` to the same value as `broker.id`.", assertEquals("You must set `node.id` to the same value as `broker.id`.",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage())
} }
@ -1559,8 +1558,8 @@ class KafkaConfigTest {
@Test @Test
def testNodeIdOrBrokerIdMustBeSetWithKraft(): Unit = { def testNodeIdOrBrokerIdMustBeSetWithKraft(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertEquals("Missing configuration `node.id` which is required when `process.roles` " + assertEquals("Missing configuration `node.id` which is required when `process.roles` " +
"is defined (i.e. when running in KRaft mode).", "is defined (i.e. when running in KRaft mode).",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage()) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage())
@ -1569,24 +1568,24 @@ class KafkaConfigTest {
@Test @Test
def testNodeIdIsInferredByBrokerIdWithKraft(): Unit = { def testNodeIdIsInferredByBrokerIdWithKraft(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KafkaConfig.BrokerIdProp, "3") props.setProperty(KafkaConfig.BrokerIdProp, "3")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals(3, config.brokerId) assertEquals(3, config.brokerId)
assertEquals(3, config.nodeId) assertEquals(3, config.nodeId)
val originals = config.originals() val originals = config.originals()
assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) assertEquals("3", originals.get(KafkaConfig.BrokerIdProp))
assertEquals("3", originals.get(KafkaConfig.NodeIdProp)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG))
} }
def kraftProps(): Properties = { def kraftProps(): Properties = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "3") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "3")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
props props
} }
@ -1599,7 +1598,7 @@ class KafkaConfigTest {
assertEquals(3, config.nodeId) assertEquals(3, config.nodeId)
val originals = config.originals() val originals = config.originals()
assertEquals("3", originals.get(KafkaConfig.BrokerIdProp)) assertEquals("3", originals.get(KafkaConfig.BrokerIdProp))
assertEquals("3", originals.get(KafkaConfig.NodeIdProp)) assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG))
} }
@Test @Test
@ -1631,11 +1630,11 @@ class KafkaConfigTest {
@Test @Test
def testEarlyStartListenersDefault(): Unit = { def testEarlyStartListenersDefault(): Unit = {
val props = new Properties() val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller") props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:8092") props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:8092")
props.setProperty(KafkaConfig.NodeIdProp, "1") props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
val config = new KafkaConfig(props) val config = new KafkaConfig(props)
assertEquals(Set("CONTROLLER"), config.earlyStartListeners.map(_.value())) assertEquals(Set("CONTROLLER"), config.earlyStartListeners.map(_.value()))
} }
@ -1698,12 +1697,12 @@ class KafkaConfigTest {
val validValue = 100 val validValue = 100
val props = new Properties() val props = new Properties()
props.putAll(kraftProps()) props.putAll(kraftProps())
props.setProperty(KafkaConfig.MetadataSnapshotMaxIntervalMsProp, validValue.toString) props.setProperty(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, validValue.toString)
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals(validValue, config.metadataSnapshotMaxIntervalMs) assertEquals(validValue, config.metadataSnapshotMaxIntervalMs)
props.setProperty(KafkaConfig.MetadataSnapshotMaxIntervalMsProp, "-1") props.setProperty(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG, "-1")
val errorMessage = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage val errorMessage = assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage
assertEquals( assertEquals(
@ -1715,17 +1714,17 @@ class KafkaConfigTest {
@Test @Test
def testMigrationEnabledZkMode(): Unit = { def testMigrationEnabledZkMode(): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort) val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort)
props.setProperty(KafkaConfig.MigrationEnabledProp, "true") props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
assertEquals( assertEquals(
"If using zookeeper.metadata.migration.enable, controller.quorum.voters must contain a parseable set of voters.", "If using zookeeper.metadata.migration.enable, controller.quorum.voters must contain a parseable set of voters.",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage) assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage)
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
assertEquals( assertEquals(
"requirement failed: controller.listener.names must not be empty when running in ZooKeeper migration mode: []", "requirement failed: controller.listener.names must not be empty when running in ZooKeeper migration mode: []",
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
// All needed configs are now set // All needed configs are now set
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
@ -1740,21 +1739,21 @@ class KafkaConfigTest {
"requirement failed: Cannot enable ZooKeeper migration without setting 'inter.broker.protocol.version' to 3.4 or higher", "requirement failed: Cannot enable ZooKeeper migration without setting 'inter.broker.protocol.version' to 3.4 or higher",
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage)
props.remove(KafkaConfig.MigrationEnabledProp) props.remove(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
assertEquals( assertEquals(
"requirement failed: controller.listener.names must be empty when not running in KRaft mode: [CONTROLLER]", "requirement failed: controller.listener.names must be empty when not running in KRaft mode: [CONTROLLER]",
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage) assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage)
props.remove(KafkaConfig.ControllerListenerNamesProp) props.remove(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)
KafkaConfig.fromProps(props) KafkaConfig.fromProps(props)
} }
@Test @Test
def testMigrationCannotBeEnabledWithJBOD(): Unit = { def testMigrationCannotBeEnabledWithJBOD(): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort, logDirCount = 2) val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort, logDirCount = 2)
props.setProperty(KafkaConfig.MigrationEnabledProp, "true") props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, MetadataVersion.IBP_3_7_IV1.version()) props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, MetadataVersion.IBP_3_7_IV1.version())
assertEquals( assertEquals(
@ -1767,7 +1766,7 @@ class KafkaConfigTest {
def testMigrationEnabledKRaftMode(): Unit = { def testMigrationEnabledKRaftMode(): Unit = {
val props = new Properties() val props = new Properties()
props.putAll(kraftProps()) props.putAll(kraftProps())
props.setProperty(KafkaConfig.MigrationEnabledProp, "true") props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
assertEquals( assertEquals(
"If using `zookeeper.metadata.migration.enable` in KRaft mode, `zookeeper.connect` must also be set.", "If using `zookeeper.metadata.migration.enable` in KRaft mode, `zookeeper.connect` must also be set.",

View File

@ -24,8 +24,9 @@ import org.apache.kafka.common.{KafkaException, Uuid}
import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.utils.Utils
import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata} import org.apache.kafka.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
@ -45,11 +46,11 @@ class KafkaRaftServerTest {
build() build()
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093")
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val metaPropertiesEnsemble = val metaPropertiesEnsemble =
invokeLoadMetaProperties(metaProperties, configProperties)._1 invokeLoadMetaProperties(metaProperties, configProperties)._1
@ -73,10 +74,10 @@ class KafkaRaftServerTest {
build() build()
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "controller") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
configProperties.put(KafkaConfig.NodeIdProp, configNodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, configNodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"$configNodeId@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$configNodeId@localhost:9092")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
assertThrows(classOf[RuntimeException], () => assertThrows(classOf[RuntimeException], () =>
invokeLoadMetaProperties(metaProperties, configProperties)) invokeLoadMetaProperties(metaProperties, configProperties))
@ -128,11 +129,11 @@ class KafkaRaftServerTest {
build()) build())
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(",")) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
assertThrows(classOf[RuntimeException], assertThrows(classOf[RuntimeException],
@ -155,12 +156,12 @@ class KafkaRaftServerTest {
// Use a regular file as an invalid log dir to trigger an IO error // Use a regular file as an invalid log dir to trigger an IO error
val invalidDir = TestUtils.tempFile("blah") val invalidDir = TestUtils.tempFile("blah")
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(KafkaConfig.MetadataLogDirProp, invalidDir.getAbsolutePath) configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, invalidDir.getAbsolutePath)
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, validDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, validDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
assertThrows(classOf[RuntimeException], assertThrows(classOf[RuntimeException],
@ -185,12 +186,12 @@ class KafkaRaftServerTest {
// Use a regular file as an invalid log dir to trigger an IO error // Use a regular file as an invalid log dir to trigger an IO error
val invalidDir = TestUtils.tempFile("blah") val invalidDir = TestUtils.tempFile("blah")
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, validDir.getAbsolutePath) configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, validDir.getAbsolutePath)
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, invalidDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, invalidDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
val (metaPropertiesEnsemble, _) = val (metaPropertiesEnsemble, _) =
@ -221,12 +222,12 @@ class KafkaRaftServerTest {
Files.createDirectory(new File(dataDir, UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition)).toPath) Files.createDirectory(new File(dataDir, UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition)).toPath)
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath) configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir.getAbsolutePath)
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, dataDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, dataDir.getAbsolutePath)
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
assertThrows(classOf[KafkaException], assertThrows(classOf[KafkaException],
@ -249,11 +250,11 @@ class KafkaRaftServerTest {
} }
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(",")) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, Seq(logDir1, logDir2).map(_.getAbsolutePath).mkString(","))
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties) val config = KafkaConfig.fromProps(configProperties)
assertThrows(classOf[RuntimeException], assertThrows(classOf[RuntimeException],
@ -272,11 +273,11 @@ class KafkaRaftServerTest {
build() build()
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093")
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
configProperties.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "3.3-IV1") configProperties.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "3.3-IV1")
val (metaPropertiesEnsemble, bootstrapMetadata) = val (metaPropertiesEnsemble, bootstrapMetadata) =
@ -303,11 +304,11 @@ class KafkaRaftServerTest {
writeMetaProperties(logDir, metaProperties) writeMetaProperties(logDir, metaProperties)
val configProperties = new Properties val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller") configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString) configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093") configProperties.put(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://127.0.0.1:9092,SSL://127.0.0.1:9093")
configProperties.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093") configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL") configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath) configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
val (metaPropertiesEnsemble, bootstrapMetadata) = val (metaPropertiesEnsemble, bootstrapMetadata) =

View File

@ -40,7 +40,8 @@ import org.apache.kafka.image.{MetadataDelta, MetadataImage}
import org.apache.kafka.metadata.LeaderRecoveryState import org.apache.kafka.metadata.LeaderRecoveryState
import org.apache.kafka.metadata.PartitionRegistration import org.apache.kafka.metadata.PartitionRegistration
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesVersion}
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.util.{MockTime, ShutdownableThread} import org.apache.kafka.server.util.{MockTime, ShutdownableThread}
import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel} import org.apache.kafka.storage.internals.log.{AppendOrigin, FetchIsolation, FetchParams, FetchPartitionData, LogConfig, LogDirFailureChannel}
@ -166,10 +167,10 @@ class ReplicaManagerConcurrencyTest extends Logging {
TestUtils.formatDirectories(immutable.Seq(logDir.getAbsolutePath), metaProperties, MetadataVersion.latestTesting(), None) TestUtils.formatDirectories(immutable.Seq(logDir.getAbsolutePath), metaProperties, MetadataVersion.latestTesting(), None)
val props = new Properties val props = new Properties
props.put(KafkaConfig.QuorumVotersProp, "100@localhost:12345") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "100@localhost:12345")
props.put(KafkaConfig.ProcessRolesProp, "broker") props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.put(KafkaConfig.NodeIdProp, localId.toString) props.put(KRaftConfigs.NODE_ID_CONFIG, localId.toString)
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath) props.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
props.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, 5000.toString) props.put(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG, 5000.toString)

View File

@ -77,7 +77,7 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils} import org.apache.kafka.metadata.properties.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion, PropertiesUtils}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.timer.MockTimer import org.apache.kafka.server.util.timer.MockTimer
import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile
import org.mockito.invocation.InvocationOnMock import org.mockito.invocation.InvocationOnMock
@ -2616,9 +2616,9 @@ class ReplicaManagerTest {
def testFullLeaderAndIsrStrayPartitions(zkMigrationEnabled: Boolean): Unit = { def testFullLeaderAndIsrStrayPartitions(zkMigrationEnabled: Boolean): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
if (zkMigrationEnabled) { if (zkMigrationEnabled) {
props.put(KafkaConfig.MigrationEnabledProp, "" + zkMigrationEnabled) props.put(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "" + zkMigrationEnabled)
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9071") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9071")
props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
config = KafkaConfig.fromProps(props) config = KafkaConfig.fromProps(props)
} }
@ -6451,9 +6451,9 @@ class ReplicaManagerTest {
timer = new MockTimer(time), timer = new MockTimer(time),
aliveBrokerIds = Seq(0, 1, 2), aliveBrokerIds = Seq(0, 1, 2),
propsModifier = props => { propsModifier = props => {
props.setProperty(KafkaConfig.MigrationEnabledProp, "true") props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(KafkaConfig.QuorumVotersProp, "1000@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT")
}, },
defaultTopicRemoteLogStorageEnable = false) defaultTopicRemoteLogStorageEnable = false)

View File

@ -38,7 +38,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer} import org.apache.kafka.common.serialization.{IntegerDeserializer, IntegerSerializer, StringDeserializer, StringSerializer}
import org.apache.kafka.common.utils.Time import org.apache.kafka.common.utils.Time
import org.apache.kafka.metadata.BrokerState import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.config.{ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs, ZkConfigs}
import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable import org.junit.jupiter.api.function.Executable
@ -147,7 +147,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
@ValueSource(strings = Array("zk", "kraft")) @ValueSource(strings = Array("zk", "kraft"))
def testCleanShutdownAfterFailedStartup(quorum: String): Unit = { def testCleanShutdownAfterFailedStartup(quorum: String): Unit = {
if (isKRaftTest()) { if (isKRaftTest()) {
propsToChangeUponRestart.setProperty(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp, "1000") propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000")
shutdownBroker() shutdownBroker()
shutdownKRaftController() shutdownKRaftController()
verifyCleanShutdownAfterFailedStartup[CancellationException] verifyCleanShutdownAfterFailedStartup[CancellationException]

View File

@ -20,6 +20,8 @@ package kafka.server
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.metadata.BrokerState import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig} import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig}
import org.apache.zookeeper.KeeperException.NodeExistsException import org.apache.zookeeper.KeeperException.NodeExistsException
@ -140,11 +142,11 @@ class ServerStartupTest extends QuorumTestHarness {
@ValueSource(booleans = Array(false, true)) @ValueSource(booleans = Array(false, true))
def testDirectoryIdsCreatedOnlyForMigration(migrationEnabled: Boolean): Unit = { def testDirectoryIdsCreatedOnlyForMigration(migrationEnabled: Boolean): Unit = {
val props = TestUtils.createBrokerConfig(1, zkConnect) val props = TestUtils.createBrokerConfig(1, zkConnect)
props.setProperty(KafkaConfig.MigrationEnabledProp, migrationEnabled.toString) props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, migrationEnabled.toString)
if (migrationEnabled) { if (migrationEnabled) {
// Create Controller properties needed when migration is enabled // Create Controller properties needed when migration is enabled
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093") props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG,
"CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT") "CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT,PLAINTEXT:PLAINTEXT")
} }

View File

@ -20,7 +20,8 @@ import java.util.Properties
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.MetricsContext import org.apache.kafka.common.metrics.MetricsContext
import org.apache.kafka.server.config.ZkConfigs import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{KRaftConfigs, ZkConfigs}
import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
@ -34,10 +35,10 @@ class ServerTest {
val clusterId = Uuid.randomUuid().toString val clusterId = Uuid.randomUuid().toString
val props = new Properties() val props = new Properties()
props.put(KafkaConfig.ProcessRolesProp, "broker") props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.put(KafkaConfig.NodeIdProp, nodeId.toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9093") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9093")
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
val context = Server.createKafkaMetricsContext(config, clusterId) val context = Server.createKafkaMetricsContext(config, clusterId)

View File

@ -31,7 +31,8 @@ import org.apache.kafka.common.utils.Utils
import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.common.metadata.UserScramCredentialRecord import org.apache.kafka.common.metadata.UserScramCredentialRecord
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.ServerLogConfigs import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertThrows, assertTrue}
import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.api.{Test, Timeout}
import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.ParameterizedTest
@ -46,10 +47,10 @@ class StorageToolTest {
private def newSelfManagedProperties() = { private def newSelfManagedProperties() = {
val properties = new Properties() val properties = new Properties()
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo,/tmp/bar") properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo,/tmp/bar")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller") properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
properties.setProperty(KafkaConfig.NodeIdProp, "2") properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9092") properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"2@localhost:9092")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT") properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
properties properties
} }
@ -62,7 +63,7 @@ class StorageToolTest {
@Test @Test
def testConfigToLogDirectoriesWithMetaLogDir(): Unit = { def testConfigToLogDirectoriesWithMetaLogDir(): Unit = {
val properties = newSelfManagedProperties() val properties = newSelfManagedProperties()
properties.setProperty(KafkaConfig.MetadataLogDirProp, "/tmp/baz") properties.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, "/tmp/baz")
val config = new KafkaConfig(properties) val config = new KafkaConfig(properties)
assertEquals(Seq("/tmp/bar", "/tmp/baz", "/tmp/foo"), assertEquals(Seq("/tmp/bar", "/tmp/baz", "/tmp/foo"),
StorageTool.configToLogDirectories(config)) StorageTool.configToLogDirectories(config))

View File

@ -74,9 +74,10 @@ import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.metadata.properties.MetaProperties import org.apache.kafka.metadata.properties.MetaProperties
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler} import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler}
import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer} import org.apache.kafka.server.authorizer.{AuthorizableRequestContext, Authorizer => JAuthorizer}
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs, QuotaConfigs, ZkConfigs} import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs, QuotaConfigs, ZkConfigs}
import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion} import org.apache.kafka.server.common.{ApiMessageAndVersion, MetadataVersion}
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
@ -333,12 +334,12 @@ object TestUtils extends Logging {
val props = new Properties val props = new Properties
props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true") props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true")
if (zkConnect == null) { if (zkConnect == null) {
props.setProperty(KafkaConfig.ServerMaxStartupTimeMsProp, TimeUnit.MINUTES.toMillis(10).toString) props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString)
props.put(KafkaConfig.NodeIdProp, nodeId.toString) props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.put(KafkaConfig.BrokerIdProp, nodeId.toString) props.put(KafkaConfig.BrokerIdProp, nodeId.toString)
props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners)
props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners) props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners)
props.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER") props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, protocolAndPorts. props.put(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, protocolAndPorts.
map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT") map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT")
} else { } else {
@ -356,12 +357,12 @@ object TestUtils extends Logging {
props.put(ServerLogConfigs.LOG_DIR_CONFIG, tempDir().getAbsolutePath) props.put(ServerLogConfigs.LOG_DIR_CONFIG, tempDir().getAbsolutePath)
} }
if (zkConnect == null) { if (zkConnect == null) {
props.put(KafkaConfig.ProcessRolesProp, "broker") props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
// Note: this is just a placeholder value for controller.quorum.voters. JUnit // Note: this is just a placeholder value for controller.quorum.voters. JUnit
// tests use random port assignment, so the controller ports are not known ahead of // tests use random port assignment, so the controller ports are not known ahead of
// time. Therefore, we ignore controller.quorum.voters and use // time. Therefore, we ignore controller.quorum.voters and use
// controllerQuorumVotersFuture instead. // controllerQuorumVotersFuture instead.
props.put(KafkaConfig.QuorumVotersProp, "1000@localhost:0") props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1000@localhost:0")
} else { } else {
props.put(ZkConfigs.ZK_CONNECT_CONFIG, zkConnect) props.put(ZkConfigs.ZK_CONNECT_CONFIG, zkConnect)
props.put(ZkConfigs.ZK_CONNECTION_TIMEOUT_MS_CONFIG, "10000") props.put(ZkConfigs.ZK_CONNECTION_TIMEOUT_MS_CONFIG, "10000")

View File

@ -29,7 +29,6 @@ import kafka.server.FetchManager;
import kafka.server.ForwardingManager; import kafka.server.ForwardingManager;
import kafka.server.KafkaApis; import kafka.server.KafkaApis;
import kafka.server.KafkaConfig; import kafka.server.KafkaConfig;
import kafka.server.KafkaConfig$;
import kafka.server.MetadataCache; import kafka.server.MetadataCache;
import kafka.server.QuotaFactory; import kafka.server.QuotaFactory;
import kafka.server.RaftSupport; import kafka.server.RaftSupport;
@ -59,8 +58,11 @@ import org.apache.kafka.coordinator.group.GroupCoordinator;
import org.apache.kafka.image.MetadataDelta; import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage; import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.MetadataProvenance; import org.apache.kafka.image.MetadataProvenance;
import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.server.common.Features; import org.apache.kafka.server.common.Features;
import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.config.KRaftConfigs;
import org.mockito.Mockito; import org.mockito.Mockito;
import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.BenchmarkMode;
@ -174,10 +176,10 @@ public class KRaftMetadataRequestBenchmark {
private KafkaApis createKafkaApis() { private KafkaApis createKafkaApis() {
Properties kafkaProps = new Properties(); Properties kafkaProps = new Properties();
kafkaProps.put(KafkaConfig$.MODULE$.NodeIdProp(), brokerId + ""); kafkaProps.put(KRaftConfigs.NODE_ID_CONFIG, brokerId + "");
kafkaProps.put(KafkaConfig$.MODULE$.ProcessRolesProp(), "broker"); kafkaProps.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker");
kafkaProps.put(KafkaConfig$.MODULE$.QuorumVotersProp(), "9000@foo:8092"); kafkaProps.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "9000@foo:8092");
kafkaProps.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(), "CONTROLLER"); kafkaProps.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER");
KafkaConfig config = new KafkaConfig(kafkaProps); KafkaConfig config = new KafkaConfig(kafkaProps);
return new KafkaApisBuilder(). return new KafkaApisBuilder().
setRequestChannel(requestChannel). setRequestChannel(requestChannel).

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.raft.QuorumConfig; import org.apache.kafka.raft.QuorumConfig;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit;
public class Defaults { public class Defaults {
/** ********* General Configuration *********/ /** ********* General Configuration *********/
@ -33,13 +32,6 @@ public class Defaults {
public static final int BACKGROUND_THREADS = 10; public static final int BACKGROUND_THREADS = 10;
public static final int QUEUED_MAX_REQUESTS = 500; public static final int QUEUED_MAX_REQUESTS = 500;
public static final int QUEUED_MAX_REQUEST_BYTES = -1; 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; public static final boolean DELETE_TOPIC_ENABLE = true;
public static final int REQUEST_TIMEOUT_MS = 30000; 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_MS = CommonClientConfigs.DEFAULT_SOCKET_CONNECTION_SETUP_TIMEOUT_MS;

View File

@ -0,0 +1,113 @@
/*
* 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 java.util.concurrent.TimeUnit;
public class KRaftConfigs {
/** KRaft mode configs */
public static final String PROCESS_ROLES_CONFIG = "process.roles";
public static final String PROCESS_ROLES_DOC = "The roles that this process plays: 'broker', 'controller', or 'broker,controller' if it is both. " +
"This configuration is only applicable for clusters in KRaft (Kafka Raft) mode (instead of ZooKeeper). Leave this config undefined or empty for ZooKeeper clusters.";
public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG = "initial.broker.registration.timeout.ms";
public static final int INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DEFAULT = 60000;
public static final String INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_DOC = "When initially registering with the controller quorum, the number of milliseconds to wait before declaring failure and exiting the broker process.";
public static final String BROKER_HEARTBEAT_INTERVAL_MS_CONFIG = "broker.heartbeat.interval.ms";
public static final int BROKER_HEARTBEAT_INTERVAL_MS_DEFAULT = 2000;
public static final String BROKER_HEARTBEAT_INTERVAL_MS_DOC = "The length of time in milliseconds between broker heartbeats. Used when running in KRaft mode.";
public static final String BROKER_SESSION_TIMEOUT_MS_CONFIG = "broker.session.timeout.ms";
public static final int BROKER_SESSION_TIMEOUT_MS_DEFAULT = 9000;
public static final String BROKER_SESSION_TIMEOUT_MS_DOC = "The length of time in milliseconds that a broker lease lasts if no heartbeats are made. Used when running in KRaft mode.";
public static final String NODE_ID_CONFIG = "node.id";
public static final int EMPTY_NODE_ID = -1;
public static final String NODE_ID_DOC = "The node ID associated with the roles this process is playing when <code>process.roles</code> is non-empty. " +
"This is required configuration when running in KRaft mode.";
public static final String METADATA_LOG_DIR_CONFIG = "metadata.log.dir";
public static final String METADATA_LOG_DIR_DOC = "This configuration determines where we put the metadata log for clusters in KRaft mode. " +
"If it is not set, the metadata log is placed in the first log directory from log.dirs.";
public static final String METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG = "metadata.log.max.snapshot.interval.ms";
public static final long METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT = TimeUnit.HOURS.toMillis(1);
public static final String METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG = "metadata.log.max.record.bytes.between.snapshots";
public static final int METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES = 20 * 1024 * 1024;
public static final String METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_DOC = "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 " +
METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES + ". To generate snapshots based on the time elapsed, see the <code>" +
METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG + "</code> configuration. The Kafka node will generate a snapshot when " +
"either the maximum time interval is reached or the maximum bytes limit is reached.";
public static final String METADATA_SNAPSHOT_MAX_INTERVAL_MS_DOC = "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 " +
"time based snapshot generation. The default value is " + METADATA_SNAPSHOT_MAX_INTERVAL_MS_DEFAULT + ". To generate " +
"snapshots based on the number of metadata bytes, see the <code>" + METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG + "</code> " +
"configuration. The Kafka node will generate a snapshot when either the maximum time interval is reached or the " +
"maximum bytes limit is reached.";
public static final String CONTROLLER_LISTENER_NAMES_CONFIG = "controller.listener.names";
public static final String CONTROLLER_LISTENER_NAMES_DOC = "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.";
public static final String SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG = "sasl.mechanism.controller.protocol";
public static final String SASL_MECHANISM_CONTROLLER_PROTOCOL_DOC = "SASL mechanism used for communication with controllers. Default is GSSAPI.";
public static final String METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG = "metadata.log.segment.min.bytes";
public static final String METADATA_LOG_SEGMENT_MIN_BYTES_DOC = "Override the minimum size for a single metadata log file. This should be used for testing only.";
public static final String METADATA_LOG_SEGMENT_BYTES_CONFIG = "metadata.log.segment.bytes";
public static final String METADATA_LOG_SEGMENT_BYTES_DOC = "The maximum size of a single metadata log file.";
public static final String METADATA_LOG_SEGMENT_MILLIS_CONFIG = "metadata.log.segment.ms";
public static final String METADATA_LOG_SEGMENT_MILLIS_DOC = "The maximum time before a new metadata log file is rolled out (in milliseconds).";
public static final String METADATA_MAX_RETENTION_BYTES_CONFIG = "metadata.max.retention.bytes";
public static final int METADATA_MAX_RETENTION_BYTES_DEFAULT = 100 * 1024 * 1024;
public static final String METADATA_MAX_RETENTION_BYTES_DOC = "The maximum combined size of the metadata log and snapshots before deleting old " +
"snapshots and log files. Since at least one snapshot must exist before any logs can be deleted, this is a soft limit.";
public static final String METADATA_MAX_RETENTION_MILLIS_CONFIG = "metadata.max.retention.ms";
public static final String METADATA_MAX_RETENTION_MILLIS_DOC = "The number of milliseconds to keep a metadata log file or snapshot before " +
"deleting it. Since at least one snapshot must exist before any logs can be deleted, this is a soft limit.";
public static final String METADATA_MAX_IDLE_INTERVAL_MS_CONFIG = "metadata.max.idle.interval.ms";
public static final int METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT = 500;
public static final String METADATA_MAX_IDLE_INTERVAL_MS_DOC = "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 " +
"are not appended to the metadata partition. The default value is " + METADATA_MAX_IDLE_INTERVAL_MS_DEFAULT;
public static final String SERVER_MAX_STARTUP_TIME_MS_CONFIG = "server.max.startup.time.ms";
public static final long SERVER_MAX_STARTUP_TIME_MS_DEFAULT = Long.MAX_VALUE;
public static final String SERVER_MAX_STARTUP_TIME_MS_DOC = "The maximum number of milliseconds we will wait for the server to come up. " +
"By default there is no limit. This should be used for testing only.";
/** ZK to KRaft Migration configs */
public static final String MIGRATION_ENABLED_CONFIG = "zookeeper.metadata.migration.enable";
public static final String MIGRATION_ENABLED_DOC = "Enable ZK to KRaft migration";
public static final String MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG = "zookeeper.metadata.migration.min.batch.size";
public static final int MIGRATION_METADATA_MIN_BATCH_SIZE_DEFAULT = 200;
public static final String MIGRATION_METADATA_MIN_BATCH_SIZE_DOC = "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft";
/** Enable eligible leader replicas configs */
public static final String ELR_ENABLED_CONFIG = "eligible.leader.replicas.enable";
public static final String ELR_ENABLED_DOC = "Enable the Eligible leader replicas";
}