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')
implementation project(':raft')
implementation project(':clients')
implementation project(':group-coordinator')
implementation project(':metadata')

View File

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

View File

@ -16,9 +16,8 @@
*/
package kafka.raft
import kafka.server.KafkaConfig
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(
logSegmentBytes: Int,
@ -35,15 +34,15 @@ final case class MetadataLogConfig(
object MetadataLogConfig {
def apply(config: AbstractConfig, maxBatchSizeInBytes: Int, maxFetchSizeInBytes: Int): MetadataLogConfig = {
new MetadataLogConfig(
config.getInt(KafkaConfig.MetadataLogSegmentBytesProp),
config.getInt(KafkaConfig.MetadataLogSegmentMinBytesProp),
config.getLong(KafkaConfig.MetadataLogSegmentMillisProp),
config.getLong(KafkaConfig.MetadataMaxRetentionBytesProp),
config.getLong(KafkaConfig.MetadataMaxRetentionMillisProp),
config.getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG),
config.getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG),
config.getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG),
config.getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG),
config.getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG),
maxBatchSizeInBytes,
maxFetchSizeInBytes,
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.SnapshotDeletionReason
import kafka.raft.KafkaMetadataLog.UnknownReason
import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMinBytesProp}
import kafka.server.{BrokerTopicStats, RequestLocal}
import kafka.utils.{CoreUtils, Logging}
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.{KafkaException, TopicPartition, Uuid}
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.snapshot.{FileRawSnapshotReader, FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals
@ -564,7 +563,7 @@ object KafkaMetadataLog extends Logging {
if (config.logSegmentBytes < config.logSegmentMinBytes) {
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) {
throw new InvalidConfigurationException(
@ -604,7 +603,7 @@ object KafkaMetadataLog extends Logging {
// Print a warning if users have overridden the internal config
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.")
}

View File

@ -21,7 +21,6 @@ import java.{lang, util}
import java.util.concurrent.TimeUnit
import java.util.{Collections, Properties}
import kafka.cluster.EndPoint
import kafka.server.KafkaConfig.ControllerListenerNamesProp
import kafka.utils.CoreUtils.parseCsvList
import kafka.utils.{CoreUtils, Logging}
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.common.{MetadataVersion, MetadataVersionValidator}
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.record.BrokerCompressionType
import org.apache.kafka.server.util.Csv
@ -109,33 +108,6 @@ object KafkaConfig {
val DeleteTopicEnableProp = "delete.topic.enable"
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 ***********/
val AuthorizerClassNameProp = "authorizer.class.name"
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 " +
"'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 ***********/
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."
@ -345,27 +278,27 @@ object KafkaConfig {
/*
* KRaft mode configs.
*/
.define(MetadataSnapshotMaxNewRecordBytesProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES, atLeast(1), HIGH, MetadataSnapshotMaxNewRecordBytesDoc)
.define(MetadataSnapshotMaxIntervalMsProp, LONG, Defaults.METADATA_SNAPSHOT_MAX_INTERVAL_MS, atLeast(0), HIGH, MetadataSnapshotMaxIntervalMsDoc)
.define(ProcessRolesProp, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, ProcessRolesDoc)
.define(NodeIdProp, INT, Defaults.EMPTY_NODE_ID, null, HIGH, NodeIdDoc)
.define(InitialBrokerRegistrationTimeoutMsProp, INT, Defaults.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS, null, MEDIUM, InitialBrokerRegistrationTimeoutMsDoc)
.define(BrokerHeartbeatIntervalMsProp, INT, Defaults.BROKER_HEARTBEAT_INTERVAL_MS, null, MEDIUM, BrokerHeartbeatIntervalMsDoc)
.define(BrokerSessionTimeoutMsProp, INT, Defaults.BROKER_SESSION_TIMEOUT_MS, null, MEDIUM, BrokerSessionTimeoutMsDoc)
.define(ControllerListenerNamesProp, STRING, null, null, HIGH, ControllerListenerNamesDoc)
.define(SaslMechanismControllerProtocolProp, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, SaslMechanismControllerProtocolDoc)
.define(MetadataLogDirProp, STRING, null, null, HIGH, MetadataLogDirDoc)
.define(MetadataLogSegmentBytesProp, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentBytesDoc)
.defineInternal(MetadataLogSegmentMinBytesProp, INT, 8 * 1024 * 1024, atLeast(Records.LOG_OVERHEAD), HIGH, MetadataLogSegmentMinBytesDoc)
.define(MetadataLogSegmentMillisProp, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, MetadataLogSegmentMillisDoc)
.define(MetadataMaxRetentionBytesProp, LONG, Defaults.METADATA_MAX_RETENTION_BYTES, null, HIGH, MetadataMaxRetentionBytesDoc)
.define(MetadataMaxRetentionMillisProp, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, MetadataMaxRetentionMillisDoc)
.define(MetadataMaxIdleIntervalMsProp, INT, Defaults.METADATA_MAX_IDLE_INTERVAL_MS, atLeast(0), LOW, MetadataMaxIdleIntervalMsDoc)
.defineInternal(ServerMaxStartupTimeMsProp, LONG, Defaults.SERVER_MAX_STARTUP_TIME_MS, atLeast(0), MEDIUM, ServerMaxStartupTimeMsDoc)
.define(MigrationEnabledProp, BOOLEAN, false, HIGH, "Enable ZK to KRaft migration")
.define(ElrEnabledProp, BOOLEAN, false, HIGH, "Enable the Eligible leader replicas")
.defineInternal(MigrationMetadataMinBatchSizeProp, INT, Defaults.MIGRATION_METADATA_MIN_BATCH_SIZE, atLeast(1),
MEDIUM, "Soft minimum batch size to use when migrating metadata from ZooKeeper to KRaft")
.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(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(KRaftConfigs.PROCESS_ROLES_CONFIG, LIST, Collections.emptyList(), ValidList.in("broker", "controller"), HIGH, KRaftConfigs.PROCESS_ROLES_DOC)
.define(KRaftConfigs.NODE_ID_CONFIG, INT, KRaftConfigs.EMPTY_NODE_ID, null, HIGH, KRaftConfigs.NODE_ID_DOC)
.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(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, INT, KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_DEFAULT, null, MEDIUM, KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_DOC)
.define(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG, INT, KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_DEFAULT, null, MEDIUM, KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_DOC)
.define(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, STRING, null, null, HIGH, KRaftConfigs.CONTROLLER_LISTENER_NAMES_DOC)
.define(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG, STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, null, HIGH, KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_DOC)
.define(KRaftConfigs.METADATA_LOG_DIR_CONFIG, STRING, null, null, HIGH, KRaftConfigs.METADATA_LOG_DIR_DOC)
.define(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG, INT, LogConfig.DEFAULT_SEGMENT_BYTES, atLeast(Records.LOG_OVERHEAD), HIGH, KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_DOC)
.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(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, LONG, LogConfig.DEFAULT_SEGMENT_MS, null, HIGH, KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_DOC)
.define(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG, LONG, KRaftConfigs.METADATA_MAX_RETENTION_BYTES_DEFAULT, null, HIGH, KRaftConfigs.METADATA_MAX_RETENTION_BYTES_DOC)
.define(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG, LONG, LogConfig.DEFAULT_RETENTION_MS, null, HIGH, KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_DOC)
.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(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(KRaftConfigs.MIGRATION_ENABLED_CONFIG, BOOLEAN, false, HIGH, KRaftConfigs.MIGRATION_ENABLED_DOC)
.define(KRaftConfigs.ELR_ENABLED_CONFIG, BOOLEAN, false, HIGH, KRaftConfigs.ELR_ENABLED_DOC)
.defineInternal(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG, INT, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DEFAULT, atLeast(1),
MEDIUM, KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_DOC)
/************* Authorizer Configuration ***********/
.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] = {
val output = new util.HashMap[Any, Any](input)
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) {
output.put(KafkaConfig.BrokerIdProp, nodeId)
} else if (brokerId != null && nodeId == null) {
output.put(KafkaConfig.NodeIdProp, brokerId)
output.put(KRaftConfigs.NODE_ID_CONFIG, brokerId)
}
output
}
@ -869,21 +802,21 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp)
val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp)
var brokerId: Int = getInt(KafkaConfig.BrokerIdProp)
val nodeId: Int = getInt(KafkaConfig.NodeIdProp)
val initialRegistrationTimeoutMs: Int = getInt(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp)
val brokerHeartbeatIntervalMs: Int = getInt(KafkaConfig.BrokerHeartbeatIntervalMsProp)
val brokerSessionTimeoutMs: Int = getInt(KafkaConfig.BrokerSessionTimeoutMsProp)
val nodeId: Int = getInt(KRaftConfigs.NODE_ID_CONFIG)
val initialRegistrationTimeoutMs: Int = getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG)
val brokerHeartbeatIntervalMs: Int = getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG)
val brokerSessionTimeoutMs: Int = getInt(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG)
def requiresZookeeper: Boolean = processRoles.isEmpty
def usesSelfManagedQuorum: Boolean = processRoles.nonEmpty
val migrationEnabled: Boolean = getBoolean(KafkaConfig.MigrationEnabledProp)
val migrationMetadataMinBatchSize: Int = getInt(KafkaConfig.MigrationMetadataMinBatchSizeProp)
val migrationEnabled: Boolean = getBoolean(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
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] = {
val roles = getList(KafkaConfig.ProcessRolesProp).asScala.map {
val roles = getList(KRaftConfigs.PROCESS_ROLES_CONFIG).asScala.map {
case "broker" => ProcessRole.BrokerRole
case "controller" => ProcessRole.ControllerRole
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
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
@ -904,17 +837,17 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
def metadataLogDir: String = {
Option(getString(KafkaConfig.MetadataLogDirProp)) match {
Option(getString(KRaftConfigs.METADATA_LOG_DIR_CONFIG)) match {
case Some(dir) => dir
case None => logDirs.head
}
}
def metadataLogSegmentBytes = getInt(KafkaConfig.MetadataLogSegmentBytesProp)
def metadataLogSegmentMillis = getLong(KafkaConfig.MetadataLogSegmentMillisProp)
def metadataRetentionBytes = getLong(KafkaConfig.MetadataMaxRetentionBytesProp)
def metadataRetentionMillis = getLong(KafkaConfig.MetadataMaxRetentionMillisProp)
val serverMaxStartupTimeMs = getLong(KafkaConfig.ServerMaxStartupTimeMsProp)
def metadataLogSegmentBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG)
def metadataLogSegmentMillis = getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG)
def metadataRetentionBytes = getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG)
def metadataRetentionMillis = getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG)
val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG)
def numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp)
def backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp)
@ -932,10 +865,10 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
/************* Metadata Configuration ***********/
val metadataSnapshotMaxNewRecordBytes = getLong(KafkaConfig.MetadataSnapshotMaxNewRecordBytesProp)
val metadataSnapshotMaxIntervalMs = getLong(KafkaConfig.MetadataSnapshotMaxIntervalMsProp)
val metadataSnapshotMaxNewRecordBytes = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG)
val metadataSnapshotMaxIntervalMs = getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG)
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
}
@ -960,7 +893,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName))
throw new ConfigException(s"${KafkaConfig.EarlyStartListenersProp} contains " +
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
}.toSet
}
@ -1291,7 +1224,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
def controllerListenerNames: Seq[String] = {
val value = Option(getString(KafkaConfig.ControllerListenerNamesProp)).getOrElse("")
val value = Option(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)).getOrElse("")
if (value.isEmpty) {
Seq.empty
} else {
@ -1302,7 +1235,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
def controllerListeners: Seq[EndPoint] =
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] = {
controlPlaneListenerName.map { listenerName =>
@ -1406,7 +1339,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
@nowarn("cat=deprecation")
private def validateValues(): Unit = {
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 (zkConnect == null) {
@ -1420,12 +1353,12 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} else {
// KRaft-based metadata quorum
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).")
}
if (migrationEnabled) {
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)
def validateNonEmptyQuorumVotersForKRaft(): Unit = {
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 = {
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 = {
@ -1463,25 +1396,25 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = {
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 = {
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 = {
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 = {
val listenerNameValues = listeners.map(_.listenerName.value).toSet
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 = {
require(advertisedListenerNames.nonEmpty,
"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)) {
// KRaft broker-only
@ -1490,24 +1423,24 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker()
// nodeId must not appear in controller.quorum.voters
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...
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...
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
controllerListenerNames.foreach { name =>
val listenerName = ListenerName.normalised(name)
if (!effectiveListenerSecurityProtocolMap.contains(listenerName)) {
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
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()
} 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
require(
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
require(
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()
validateControllerListenerExistsForKRaftController()
@ -1541,7 +1474,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
if (migrationEnabled) {
validateNonEmptyQuorumVotersForMigration()
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 " +
s"'${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG}' to 3.4 or higher")
if (logDirs.size > 1) {
@ -1552,7 +1485,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
} else {
// controller listener names must be empty when not in KRaft mode
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()
}

View File

@ -122,7 +122,7 @@ object KafkaRaftServer {
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
* 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.server.common.{Features, MetadataVersion}
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.util.{CommandDefaultOptions, CommandLineUtils, ShutdownableThread}
import org.apache.kafka.snapshot.SnapshotReader
@ -447,7 +448,7 @@ object TestRaftServer extends Logging {
// 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.
serverProps.put(KafkaConfig.ProcessRolesProp, "controller")
serverProps.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
val config = KafkaConfig.fromProps(serverProps, doLog = false)
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.MetadataProvenance;
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.AuthorizationResult;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.config.KRaftConfigs;
import org.junit.jupiter.api.Test;
import java.net.InetAddress;
@ -550,11 +553,11 @@ class DescribeTopicPartitionsRequestHandlerTest {
1,
(short) 1,
false);
properties.put(KafkaConfig.NodeIdProp(), Integer.toString(brokerId));
properties.put(KafkaConfig.ProcessRolesProp(), "broker");
properties.put(KRaftConfigs.NODE_ID_CONFIG, Integer.toString(brokerId));
properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker");
int voterId = brokerId + 1;
properties.put(KafkaConfig.QuorumVotersProp(), voterId + "@localhost:9093");
properties.put(KafkaConfig.ControllerListenerNamesProp(), "SSL");
properties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, voterId + "@localhost:9093");
properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL");
TestUtils.setIbpAndMessageFormatVersions(properties, MetadataVersion.latestProduction());
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.raft.QuorumConfig;
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.MockFaultHandler;
import org.apache.kafka.storage.internals.log.CleanerConfig;
@ -158,17 +159,17 @@ public class KafkaClusterTestKit implements AutoCloseable {
ControllerNode controllerNode = nodes.controllerNodes().get(node.id());
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)));
props.put(KafkaConfig$.MODULE$.ProcessRolesProp(), roles(node.id()));
props.put(KafkaConfig$.MODULE$.NodeIdProp(),
props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, roles(node.id()));
props.put(KRaftConfigs.NODE_ID_CONFIG,
Integer.toString(node.id()));
// In combined mode, always prefer the metadata log directory of the controller node.
if (controllerNode != null) {
props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
props.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG,
controllerNode.metadataDirectory());
} else {
props.put(KafkaConfig$.MODULE$.MetadataLogDirProp(),
props.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG,
node.metadataDirectory());
}
if (brokerNode != null) {
@ -185,7 +186,7 @@ public class KafkaClusterTestKit implements AutoCloseable {
props.put(SocketServerConfigs.LISTENERS_CONFIG, listeners(node.id()));
props.put(INTER_BROKER_LISTENER_NAME_CONFIG,
nodes.interBrokerListenerName().value());
props.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(),
props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG,
"CONTROLLER");
// 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

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.coordinator.group.GroupCoordinatorConfig
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 scala.collection.mutable
@ -68,14 +68,14 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
configureListeners(cfgs)
modifyConfigs(cfgs)
if (isZkMigrationTest()) {
cfgs.foreach(_.setProperty(KafkaConfig.MigrationEnabledProp, "true"))
cfgs.foreach(_.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true"))
}
if (isNewGroupCoordinatorEnabled()) {
cfgs.foreach(_.setProperty(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG, "true"))
}
if(isKRaftTest()) {
cfgs.foreach(_.setProperty(KafkaConfig.MetadataLogDirProp, TestUtils.tempDir().getAbsolutePath))
cfgs.foreach(_.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, TestUtils.tempDir().getAbsolutePath))
}
insertControllerListenersIfNeeded(cfgs)
@ -106,7 +106,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
props.foreach { config =>
// 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 toAdd = config.getProperty(KafkaConfig.ControllerListenerNamesProp, "").split(",").filter(
val toAdd = config.getProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "").split(",").filter(
e => !securityPairs.exists(_.startsWith(s"$e:")))
if (toAdd.nonEmpty) {
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.server.authorizer._
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.quota
import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaType}
@ -1248,8 +1249,8 @@ class KRaftClusterTest {
new TestKitNodes.Builder().
setNumBrokerNodes(3).
setNumControllerNodes(1).build()).
setConfigProp(KafkaConfig.BrokerHeartbeatIntervalMsProp, 10.toString).
setConfigProp(KafkaConfig.BrokerSessionTimeoutMsProp, 1000.toString).
setConfigProp(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, 10.toString).
setConfigProp(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG, 1000.toString).
build()
try {
cluster.format()

View File

@ -26,7 +26,7 @@ import org.apache.kafka.common.Uuid
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
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.extension.ExtendWith
import org.junit.jupiter.api.{Tag, Timeout}
@ -62,7 +62,7 @@ class KafkaServerKRaftRegistrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -71,9 +71,9 @@ class KafkaServerKRaftRegistrationTest {
val readyFuture = kraftCluster.controllers().values().asScala.head.controller.waitForReadyBrokers(3)
// 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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()
@ -100,7 +100,7 @@ class KafkaServerKRaftRegistrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -108,9 +108,9 @@ class KafkaServerKRaftRegistrationTest {
kraftCluster.startup()
// 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(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")
assertThrows(classOf[IllegalArgumentException], () => zkCluster.rollingBrokerRestart())
} 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.{MetaProperties, MetaPropertiesEnsemble, MetaPropertiesVersion}
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.config.KRaftConfigs
import org.apache.kafka.server.fault.{FaultHandler, MockFaultHandler}
import org.apache.zookeeper.client.ZKClientConfig
import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper}
@ -89,7 +90,7 @@ class KRaftQuorumImplementation(
val controllerServer: ControllerServer,
val faultHandlerFactory: FaultHandlerFactory,
val metadataDir: File,
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, AddressSpec]],
val controllerQuorumVotersFuture: CompletableFuture[util.Map[Integer, QuorumConfig.AddressSpec]],
val clusterId: String,
val log: Logging,
val faultHandler: FaultHandler
@ -321,13 +322,13 @@ abstract class QuorumTestHarness extends Logging {
}
val props = propsList.head
props.putAll(overridingProps)
props.setProperty(KafkaConfig.ServerMaxStartupTimeMsProp, TimeUnit.MINUTES.toMillis(10).toString)
props.setProperty(KafkaConfig.ProcessRolesProp, "controller")
props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString)
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(KafkaConfig.UnstableMetadataVersionsEnableProp, "true")
if (props.getProperty(KafkaConfig.NodeIdProp) == null) {
props.setProperty(KafkaConfig.NodeIdProp, "1000")
if (props.getProperty(KRaftConfigs.NODE_ID_CONFIG) == null) {
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 metaProperties = new MetaProperties.Builder().
setVersion(MetaPropertiesVersion.V1).
@ -347,14 +348,14 @@ abstract class QuorumTestHarness extends Logging {
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
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, s"CONTROLLER:$proto")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"CONTROLLER://localhost:0")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:0")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:0")
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().
addMetadataLogDir(metadataDir.getAbsolutePath).
load()
@ -380,7 +381,7 @@ abstract class QuorumTestHarness extends Logging {
controllerQuorumVotersFuture.completeExceptionally(e)
} else {
controllerQuorumVotersFuture.complete(Collections.singletonMap(nodeId,
new InetAddressSpec(new InetSocketAddress("localhost", port))))
new QuorumConfig.InetAddressSpec(new InetSocketAddress("localhost", port))))
}
})
controllerServer.startup()

View File

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

View File

@ -16,7 +16,7 @@
*/
package kafka.zk
import kafka.server.{KRaftCachedControllerId, KafkaConfig}
import kafka.server.KRaftCachedControllerId
import kafka.test.{ClusterConfig, ClusterGenerator, ClusterInstance}
import kafka.test.annotation.{AutoStart, ClusterConfigProperty, ClusterTemplate, ClusterTest, Type}
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.server.ControllerRequestCompletionHandler
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.{Assumptions, Timeout}
import org.junit.jupiter.api.extension.ExtendWith
@ -177,7 +177,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -187,9 +187,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart() // This would throw if authorizers weren't allowed
zkCluster.waitForReadyBrokers()
@ -309,7 +309,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -319,9 +319,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
@ -443,7 +443,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -453,9 +453,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()
@ -507,7 +507,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -517,9 +517,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart() // This would throw if authorizers weren't allowed
zkCluster.waitForReadyBrokers()
@ -589,7 +589,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -602,9 +602,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()
@ -657,7 +657,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -667,9 +667,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()
@ -717,7 +717,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -727,9 +727,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()
@ -792,7 +792,7 @@ class ZkMigrationIntegrationTest {
setClusterId(Uuid.fromString(clusterId)).
setNumBrokerNodes(0).
setNumControllerNodes(1).build())
.setConfigProp(KafkaConfig.MigrationEnabledProp, "true")
.setConfigProp(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
.setConfigProp(ZkConfigs.ZK_CONNECT_CONFIG, zkCluster.asInstanceOf[ZkClusterInstance].getUnderlying.zkConnect)
.build()
try {
@ -802,9 +802,9 @@ class ZkMigrationIntegrationTest {
// Enable migration configs and restart brokers
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(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.rollingBrokerRestart()
zkCluster.waitForReadyBrokers()

View File

@ -17,7 +17,6 @@
package kafka.raft
import kafka.log.UnifiedLog
import kafka.server.KafkaConfig.{MetadataLogSegmentBytesProp, MetadataLogSegmentMillisProp, MetadataLogSegmentMinBytesProp, NodeIdProp, ProcessRolesProp, QuorumVotersProp}
import kafka.server.{KafkaConfig, KafkaRaftServer}
import kafka.utils.TestUtils
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.internals.BatchBuilder
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.snapshot.{FileRawSnapshotWriter, RawSnapshotReader, RawSnapshotWriter, SnapshotPath, Snapshots}
import org.apache.kafka.storage.internals.log.{LogConfig, LogStartOffsetIncrementReason}
@ -62,19 +61,19 @@ final class KafkaMetadataLogTest {
@Test
def testConfig(): Unit = {
val props = new Properties()
props.put(ProcessRolesProp, util.Arrays.asList("broker"))
props.put(QuorumVotersProp, "1@localhost:9093")
props.put(NodeIdProp, Int.box(2))
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.put(MetadataLogSegmentBytesProp, Int.box(10240))
props.put(MetadataLogSegmentMillisProp, Int.box(10 * 1024))
props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, util.Arrays.asList("broker"))
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
props.put(KRaftConfigs.NODE_ID_CONFIG, Int.box(2))
props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.put(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG, Int.box(10240))
props.put(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG, Int.box(10 * 1024))
assertThrows(classOf[InvalidConfigurationException], () => {
val kafkaConfig = KafkaConfig.fromProps(props)
val metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
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 metadataConfig = MetadataLogConfig(kafkaConfig, KafkaRaftClient.MAX_BATCH_SIZE_BYTES, KafkaRaftClient.MAX_FETCH_SIZE_BYTES)
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.internals.FatalExitError
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.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.api.Assertions._
@ -86,15 +88,15 @@ class KafkaConfigTest {
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
val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile,
"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
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile)
}
@ -102,15 +104,15 @@ class KafkaConfigTest {
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
val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile,
"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
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile)
}
@ -118,24 +120,24 @@ class KafkaConfigTest {
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
val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile,
"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
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "2")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
KafkaConfig.fromProps(propertiesFile)
}
@Test
def testIsKRaftCombinedMode(): Unit = {
val propertiesFile = new Properties
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
setListenerProps(propertiesFile)
val config = KafkaConfig.fromProps(propertiesFile)
assertTrue(config.isKRaftCombinedMode)
@ -145,32 +147,32 @@ class KafkaConfigTest {
def testMustContainQuorumVotersIfUsingProcessRoles(): Unit = {
// 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
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "controller,broker")
propertiesFile.setProperty(KafkaConfig.NodeIdProp, "1")
propertiesFile.setProperty(KafkaConfig.QuorumVotersProp, "")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller,broker")
propertiesFile.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
propertiesFile.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "")
setListenerProps(propertiesFile)
assertBadConfigContainingMessage(propertiesFile,
"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
propertiesFile.setProperty(KafkaConfig.ProcessRolesProp, "")
propertiesFile.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "")
assertBadConfigContainingMessage(propertiesFile,
"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)
propertiesFile.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
propertiesFile.setProperty(KafkaConfig.ControllerListenerNamesProp, "")
propertiesFile.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "")
KafkaConfig.fromProps(propertiesFile)
}
private def setListenerProps(props: Properties): Unit = {
val hasBrokerRole = props.getProperty(KafkaConfig.ProcessRolesProp).contains("broker")
val hasControllerRole = props.getProperty(KafkaConfig.ProcessRolesProp).contains("controller")
val hasBrokerRole = props.getProperty(KRaftConfigs.PROCESS_ROLES_CONFIG).contains("broker")
val hasControllerRole = props.getProperty(KRaftConfigs.PROCESS_ROLES_CONFIG).contains("controller")
val controllerListener = "SASL_PLAINTEXT://localhost:9092"
val brokerListener = "PLAINTEXT://localhost:9093"
if (hasBrokerRole || hasControllerRole) { // KRaft
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SASL_PLAINTEXT")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SASL_PLAINTEXT")
if (hasBrokerRole && hasControllerRole) {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"$brokerListener,$controllerListener")
} 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.network.SocketServerConfigs
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.config.ZkConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
@ -55,11 +54,11 @@ class RaftManagerTest {
}
if (migrationEnabled) {
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(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
}
props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
@ -78,23 +77,23 @@ class RaftManagerTest {
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, value.toString)
}
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(KafkaConfig.NodeIdProp, nodeId.toString)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, processRoles.mkString(","))
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
if (processRoles.contains(ProcessRole.BrokerRole)) {
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "PLAINTEXT")
if (processRoles.contains(ProcessRole.ControllerRole)) { // co-located
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
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
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)

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.requests.{AbstractRequest, AbstractResponse, BrokerHeartbeatRequest, BrokerHeartbeatResponse, BrokerRegistrationRequest, BrokerRegistrationResponse}
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.{Test, Timeout}
@ -37,12 +38,12 @@ class BrokerLifecycleManagerTest {
def configProperties = {
val properties = new Properties()
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo")
properties.setProperty(KafkaConfig.ProcessRolesProp, "broker")
properties.setProperty(KafkaConfig.NodeIdProp, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9093")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
properties.setProperty(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp, "300000")
properties.setProperty(KafkaConfig.BrokerHeartbeatIntervalMsProp, "100")
properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"2@localhost:9093")
properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
properties.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "300000")
properties.setProperty(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG, "100")
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.{Controller, ControllerRequestContext, ResultOrError}
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.common.{ApiMessageAndVersion, Features, MetadataVersion, ProducerIdsBlock}
import org.apache.kafka.server.config.KRaftConfigs
import org.apache.kafka.server.util.FutureUtils
import org.apache.kafka.storage.internals.log.CleanerConfig
import org.junit.jupiter.api.Assertions._
@ -151,10 +153,10 @@ class ControllerApisTest {
controller: Controller,
props: Properties = new Properties(),
throttle: Boolean = false): ControllerApis = {
props.put(KafkaConfig.NodeIdProp, nodeId: java.lang.Integer)
props.put(KafkaConfig.ProcessRolesProp, "controller")
props.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT")
props.put(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9092")
props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId: java.lang.Integer)
props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9092")
new ControllerApis(
requestChannel,
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.metadata.{ListenerInfo, RecordTestUtils, VersionRange}
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.config.ServerLogConfigs
import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.{Test, Timeout}
@ -48,12 +48,12 @@ class ControllerRegistrationManagerTest {
private def configProperties = {
val properties = new Properties()
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.LISTENERS_CONFIG, s"CONTROLLER://localhost:8001")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
properties.setProperty(KafkaConfig.NodeIdProp, "1")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"1@localhost:8000,2@localhost:5000,3@localhost:7000")
properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"1@localhost:8000,2@localhost:5000,3@localhost:7000")
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.metrics.{JmxReporter, Metrics}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.PasswordEncoderConfigs
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.metrics.KafkaYammerMetrics
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.test.MockMetricsReporter
import org.junit.jupiter.api.Assertions._
@ -518,10 +518,10 @@ class DynamicBrokerConfigTest {
enableControlledShutdown = true,
enableDeleteTopic = true,
port)
retval.put(KafkaConfig.ProcessRolesProp, "broker,controller")
retval.put(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
retval.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,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(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0")
retval.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId}@localhost:0")
retval
}
@ -562,12 +562,12 @@ class DynamicBrokerConfigTest {
enableDeleteTopic = true,
port
)
retval.put(KafkaConfig.ProcessRolesProp, "controller")
retval.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
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(KafkaConfig.QuorumVotersProp, s"${nodeId}@localhost:0")
retval.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId}@localhost:0")
retval
}
@ -713,11 +713,11 @@ class DynamicBrokerConfigTest {
def testNonInternalValuesDoesNotExposeInternalConfigs(): Unit = {
val props = new Properties()
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)
assertFalse(config.nonInternalValues.containsKey(KafkaConfig.MetadataLogSegmentMinBytesProp))
assertFalse(config.nonInternalValues.containsKey(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG))
config.updateCurrentConfig(new KafkaConfig(props))
assertFalse(config.nonInternalValues.containsKey(KafkaConfig.MetadataLogSegmentMinBytesProp))
assertFalse(config.nonInternalValues.containsKey(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG))
}
@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.coordinator.transaction.TransactionLogConfigs
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.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.common.MetadataVersion.{IBP_0_10_2_IV0, IBP_2_2_IV1}
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.util.{FutureUtils, MockTime}
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 = {
val properties = if (raftSupport) {
val properties = TestUtils.createBrokerConfig(brokerId, "")
properties.put(KafkaConfig.NodeIdProp, brokerId.toString)
properties.put(KafkaConfig.ProcessRolesProp, "broker")
properties.put(KRaftConfigs.NODE_ID_CONFIG, brokerId.toString)
properties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
val voterId = brokerId + 1
properties.put(KafkaConfig.QuorumVotersProp, s"$voterId@localhost:9093")
properties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
properties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$voterId@localhost:9093")
properties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
properties
} else {
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.record.{CompressionType, Records}
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.Test
@ -40,10 +40,9 @@ import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, Transact
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.security.PasswordEncoderConfigs
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.{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.storage.internals.log.CleanerConfig
import org.junit.jupiter.api.function.Executable
@ -286,11 +285,11 @@ class KafkaConfigTest {
@Test
def testControllerListenerNames(): Unit = {
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(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:5000")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:5000")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SASL_SSL")
val serverConfig = KafkaConfig.fromProps(props)
@ -305,11 +304,11 @@ class KafkaConfigTest {
@Test
def testControlPlaneListenerNameNotAllowedWithKRaft(): Unit = {
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(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
props.setProperty(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, "SSL")
assertFalse(isValidKafkaConfig(props))
@ -322,50 +321,50 @@ class KafkaConfigTest {
@Test
def testControllerListenerDefinedForKRaftController(): Unit = {
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(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
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")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
KafkaConfig.fromProps(props)
// confirm that redirecting via listener.security.protocol.map is acceptable
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:SSL")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
KafkaConfig.fromProps(props)
}
@Test
def testControllerListenerDefinedForKRaftBroker(): Unit = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props))
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)
// confirm that redirecting via listener.security.protocol.map is acceptable
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)
}
@Test
def testPortInQuorumVotersNotRequiredToMatchFirstControllerListenerPortForThisKRaftController(): Unit = {
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(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093,3@anotherhost:9094")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL,SASL_SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093,3@anotherhost:9094")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL,SASL_SSL")
KafkaConfig.fromProps(props)
// 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")
KafkaConfig.fromProps(props)
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)
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)
}
@Test
def testSeparateControllerListenerDefinedForKRaftBrokerController(): Unit = {
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(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props))
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
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(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
KafkaConfig.fromProps(props)
}
@Test
def testControllerListenerNameMapsToPlaintextByDefaultForKRaft(): Unit = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
val controllerListenerName = new ListenerName("CONTROLLER")
assertEquals(Some(SecurityProtocol.PLAINTEXT),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName))
// 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"
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
// 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")
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
props.remove(SocketServerConfigs.LISTENERS_CONFIG)
@ -434,7 +433,7 @@ class KafkaConfigTest {
assertBadConfigContainingMessage(props, controllerNotFoundInMapMessage)
// 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.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
assertEquals(Some(SecurityProtocol.SSL),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("SSL")))
}
@ -442,11 +441,11 @@ class KafkaConfigTest {
@Test
def testMultipleControllerListenerNamesMapToPlaintextByDefaultForKRaft(): Unit = {
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(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER1,CONTROLLER2")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9092")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER1,CONTROLLER2")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9092")
assertEquals(Some(SecurityProtocol.PLAINTEXT),
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER1")))
assertEquals(Some(SecurityProtocol.PLAINTEXT),
@ -811,18 +810,18 @@ class KafkaConfigTest {
case KafkaConfig.ConnectionSetupTimeoutMaxMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
// KRaft mode configs
case KafkaConfig.ProcessRolesProp => // ignore
case KafkaConfig.InitialBrokerRegistrationTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.BrokerHeartbeatIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.BrokerSessionTimeoutMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.NodeIdProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataLogDirProp => // ignore string
case KafkaConfig.MetadataLogSegmentBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataLogSegmentMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataMaxRetentionBytesProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.MetadataMaxRetentionMillisProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.ControllerListenerNamesProp => // ignore string
case KafkaConfig.MetadataMaxIdleIntervalMsProp => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.PROCESS_ROLES_CONFIG => // ignore
case KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.NODE_ID_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.METADATA_LOG_DIR_CONFIG => // ignore string
case KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG => // ignore string
case KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number")
case KafkaConfig.AuthorizerClassNameProp => //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
//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_ENABLED_MECHANISMS_CONFIG =>
case KafkaSecurityConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS_CONFIG =>
@ -1221,12 +1220,12 @@ class KafkaConfigTest {
@Test
def testDistinctControllerAndAdvertisedListenersAllowedForKRaftBroker(): Unit = {
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.ADVERTISED_LISTENERS_CONFIG, "PLAINTEXT://A:9092,SSL://B:9093") // explicitly setting it in KRaft
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SASL_SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "3@localhost:9094")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SASL_SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3@localhost:9094")
// invalid due to extra listener also appearing in controller listeners
assertBadConfigContainingMessage(props,
@ -1245,14 +1244,14 @@ class KafkaConfigTest {
@Test
def testControllerListenersCannotBeAdvertisedForKRaftBroker(): Unit = {
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"
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, listeners)
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, listeners) // explicitly setting it in KRaft
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "SASL_SSL")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT,SSL")
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT,SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
assertBadConfigContainingMessage(props,
"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 props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, correctListeners)
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, incorrectListeners)
props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames)
props.setProperty(KafkaConfig.NodeIdProp, "2")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9092")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9092")
var expectedExceptionContainsText = "The advertised.listeners config must be empty when process.roles=controller"
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
@ -1298,13 +1297,13 @@ class KafkaConfigTest {
// Invalid if listeners doesn't contain every name in controller.listener.names
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'
|configuration when running the KRaft controller role""".stripMargin.replaceAll("\n", " ")
assertBadConfigContainingMessage(props, expectedExceptionContainsText)
// Valid now
props.setProperty(KafkaConfig.ControllerListenerNamesProp, correctControllerListenerNames)
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, correctControllerListenerNames)
KafkaConfig.fromProps(props)
}
@ -1336,32 +1335,32 @@ class KafkaConfigTest {
private def assertInvalidQuorumVoters(value: String): Unit = {
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))
}
@Test
def testValidQuorumVotersConfig(): Unit = {
val expected = new util.HashMap[Integer, AddressSpec]()
val expected = new util.HashMap[Integer, QuorumConfig.AddressSpec]()
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)
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)
expected.clear()
expected.put(1, new InetAddressSpec(new InetSocketAddress("kafka1", 9092)))
expected.put(2, new InetAddressSpec(new InetSocketAddress("kafka2", 9092)))
expected.put(3, new InetAddressSpec(new InetSocketAddress("kafka3", 9092)))
expected.put(1, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka1", 9092)))
expected.put(2, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka2", 9092)))
expected.put(3, new QuorumConfig.InetAddressSpec(new InetSocketAddress("kafka3", 9092)))
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)
props.setProperty(KafkaConfig.QuorumVotersProp, value)
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, value)
val raftConfig = new QuorumConfig(KafkaConfig.fromProps(props))
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)
val largeBrokerId = 2000
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(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KafkaConfig.NodeIdProp, largeBrokerId.toString)
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, largeBrokerId.toString)
KafkaConfig.fromProps(props)
}
@ -1385,7 +1384,7 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedBrokerCaseWithAutoGenEnabled(): Unit = {
// -1 is the default for both node.id and broker.id
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
assertFalse(isValidKafkaConfig(props))
}
@ -1393,7 +1392,7 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedControllerCaseWithAutoGenEnabled(): Unit = {
// -1 is the default for both node.id and broker.id
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
assertFalse(isValidKafkaConfig(props))
}
@ -1401,9 +1400,9 @@ class KafkaConfigTest {
def testRejectsNegativeNodeIdForRaftBasedCaseWithAutoGenDisabled(): Unit = {
// -1 is the default for both node.id and broker.id
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "false")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertFalse(isValidKafkaConfig(props))
}
@ -1438,7 +1437,7 @@ class KafkaConfigTest {
val listeners = "PLAINTEXT://A:9092,SSL://B:9093,SASL_SSL://C:9094"
props.setProperty(SocketServerConfigs.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)
assertFalse(isValidKafkaConfig(props))
}
@ -1469,7 +1468,7 @@ class KafkaConfigTest {
@Test
def testZookeeperConnectRequiredIfEmptyProcessRoles(): Unit = {
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")
assertFalse(isValidKafkaConfig(props))
}
@ -1477,11 +1476,11 @@ class KafkaConfigTest {
@Test
def testZookeeperConnectNotRequiredIfNonEmptyProcessRoles(): Unit = {
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(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props)
}
@ -1491,12 +1490,12 @@ class KafkaConfigTest {
val dataDir = "/path/to/data/dir"
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KafkaConfig.MetadataLogDirProp, metadataDir)
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir)
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, dataDir)
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props)
val config = KafkaConfig.fromProps(props)
@ -1510,11 +1509,11 @@ class KafkaConfigTest {
val dataDir2 = "/path/to/data/dir/2"
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.setProperty(ServerLogConfigs.LOG_DIR_CONFIG, s"$dataDir1,$dataDir2")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
KafkaConfig.fromProps(props)
val config = KafkaConfig.fromProps(props)
@ -1533,17 +1532,17 @@ class KafkaConfigTest {
input.put(KafkaConfig.BrokerIdProp, "4")
val expectedOutput = new util.HashMap[String, String]()
expectedOutput.put(KafkaConfig.BrokerIdProp, "4")
expectedOutput.put(KafkaConfig.NodeIdProp, "4")
expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4")
assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input))
}
@Test
def testPopulateSynonymsOnMapWithoutBrokerId(): Unit = {
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]()
expectedOutput.put(KafkaConfig.BrokerIdProp, "4")
expectedOutput.put(KafkaConfig.NodeIdProp, "4")
expectedOutput.put(KRaftConfigs.NODE_ID_CONFIG, "4")
assertEquals(expectedOutput, KafkaConfig.populateSynonyms(input))
}
@ -1551,7 +1550,7 @@ class KafkaConfigTest {
def testNodeIdMustNotBeDifferentThanBrokerId(): Unit = {
val props = new Properties()
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`.",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage())
}
@ -1559,8 +1558,8 @@ class KafkaConfigTest {
@Test
def testNodeIdOrBrokerIdMustBeSetWithKraft(): Unit = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.QuorumVotersProp, "2@localhost:9093")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "2@localhost:9093")
assertEquals("Missing configuration `node.id` which is required when `process.roles` " +
"is defined (i.e. when running in KRaft mode).",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage())
@ -1569,24 +1568,24 @@ class KafkaConfigTest {
@Test
def testNodeIdIsInferredByBrokerIdWithKraft(): Unit = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
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)
assertEquals(3, config.brokerId)
assertEquals(3, config.nodeId)
val originals = config.originals()
assertEquals("3", originals.get(KafkaConfig.BrokerIdProp))
assertEquals("3", originals.get(KafkaConfig.NodeIdProp))
assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG))
}
def kraftProps(): Properties = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "broker")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KafkaConfig.NodeIdProp, "3")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9093")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "3")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
props
}
@ -1599,7 +1598,7 @@ class KafkaConfigTest {
assertEquals(3, config.nodeId)
val originals = config.originals()
assertEquals("3", originals.get(KafkaConfig.BrokerIdProp))
assertEquals("3", originals.get(KafkaConfig.NodeIdProp))
assertEquals("3", originals.get(KRaftConfigs.NODE_ID_CONFIG))
}
@Test
@ -1631,11 +1630,11 @@ class KafkaConfigTest {
@Test
def testEarlyStartListenersDefault(): Unit = {
val props = new Properties()
props.setProperty(KafkaConfig.ProcessRolesProp, "controller")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "CONTROLLER://:8092")
props.setProperty(KafkaConfig.NodeIdProp, "1")
props.setProperty(KafkaConfig.QuorumVotersProp, "1@localhost:9093")
props.setProperty(KRaftConfigs.NODE_ID_CONFIG, "1")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1@localhost:9093")
val config = new KafkaConfig(props)
assertEquals(Set("CONTROLLER"), config.earlyStartListeners.map(_.value()))
}
@ -1698,12 +1697,12 @@ class KafkaConfigTest {
val validValue = 100
val props = new Properties()
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)
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
assertEquals(
@ -1715,17 +1714,17 @@ class KafkaConfigTest {
@Test
def testMigrationEnabledZkMode(): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort)
props.setProperty(KafkaConfig.MigrationEnabledProp, "true")
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
assertEquals(
"If using zookeeper.metadata.migration.enable, controller.quorum.voters must contain a parseable set of voters.",
assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props)).getMessage)
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
assertEquals(
"requirement failed: controller.listener.names must not be empty when running in ZooKeeper migration mode: []",
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
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",
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage)
props.remove(KafkaConfig.MigrationEnabledProp)
props.remove(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
assertEquals(
"requirement failed: controller.listener.names must be empty when not running in KRaft mode: [CONTROLLER]",
assertThrows(classOf[IllegalArgumentException], () => KafkaConfig.fromProps(props)).getMessage)
props.remove(KafkaConfig.ControllerListenerNamesProp)
props.remove(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)
KafkaConfig.fromProps(props)
}
@Test
def testMigrationCannotBeEnabledWithJBOD(): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, port = TestUtils.MockZkPort, logDirCount = 2)
props.setProperty(KafkaConfig.MigrationEnabledProp, "true")
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, MetadataVersion.IBP_3_7_IV1.version())
assertEquals(
@ -1767,7 +1766,7 @@ class KafkaConfigTest {
def testMigrationEnabledKRaftMode(): Unit = {
val props = new Properties()
props.putAll(kraftProps())
props.setProperty(KafkaConfig.MigrationEnabledProp, "true")
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
assertEquals(
"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.metadata.bootstrap.{BootstrapDirectory, BootstrapMetadata}
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.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.config.{KRaftConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.test.TestUtils
import org.junit.jupiter.api.Assertions._
@ -45,11 +46,11 @@ class KafkaRaftServerTest {
build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
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(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val metaPropertiesEnsemble =
invokeLoadMetaProperties(metaProperties, configProperties)._1
@ -73,10 +74,10 @@ class KafkaRaftServerTest {
build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "controller")
configProperties.put(KafkaConfig.NodeIdProp, configNodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"$configNodeId@localhost:9092")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT")
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, configNodeId.toString)
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$configNodeId@localhost:9092")
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
assertThrows(classOf[RuntimeException], () =>
invokeLoadMetaProperties(metaProperties, configProperties))
@ -128,11 +129,11 @@ class KafkaRaftServerTest {
build())
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092")
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
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(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(configProperties)
assertThrows(classOf[RuntimeException],
@ -155,12 +156,12 @@ class KafkaRaftServerTest {
// Use a regular file as an invalid log dir to trigger an IO error
val invalidDir = TestUtils.tempFile("blah")
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.MetadataLogDirProp, invalidDir.getAbsolutePath)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, invalidDir.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)
assertThrows(classOf[RuntimeException],
@ -185,12 +186,12 @@ class KafkaRaftServerTest {
// Use a regular file as an invalid log dir to trigger an IO error
val invalidDir = TestUtils.tempFile("blah")
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, validDir.getAbsolutePath)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, validDir.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 (metaPropertiesEnsemble, _) =
@ -221,12 +222,12 @@ class KafkaRaftServerTest {
Files.createDirectory(new File(dataDir, UnifiedLog.logDirName(KafkaRaftServer.MetadataPartition)).toPath)
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.MetadataLogDirProp, metadataDir.getAbsolutePath)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KRaftConfigs.METADATA_LOG_DIR_CONFIG, metadataDir.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)
assertThrows(classOf[KafkaException],
@ -249,11 +250,11 @@ class KafkaRaftServerTest {
}
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker")
configProperties.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9092")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9092")
configProperties.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
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)
assertThrows(classOf[RuntimeException],
@ -272,11 +273,11 @@ class KafkaRaftServerTest {
build()
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
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(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
configProperties.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "3.3-IV1")
val (metaPropertiesEnsemble, bootstrapMetadata) =
@ -303,11 +304,11 @@ class KafkaRaftServerTest {
writeMetaProperties(logDir, metaProperties)
val configProperties = new Properties
configProperties.put(KafkaConfig.ProcessRolesProp, "broker,controller")
configProperties.put(KafkaConfig.NodeIdProp, nodeId.toString)
configProperties.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker,controller")
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(KafkaConfig.QuorumVotersProp, s"$nodeId@localhost:9093")
configProperties.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
configProperties.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"$nodeId@localhost:9093")
configProperties.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
configProperties.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
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.PartitionRegistration
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.util.{MockTime, ShutdownableThread}
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)
val props = new Properties
props.put(KafkaConfig.QuorumVotersProp, "100@localhost:12345")
props.put(KafkaConfig.ProcessRolesProp, "broker")
props.put(KafkaConfig.NodeIdProp, localId.toString)
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "100@localhost:12345")
props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.put(KRaftConfigs.NODE_ID_CONFIG, localId.toString)
props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
props.put(ServerLogConfigs.LOG_DIR_CONFIG, logDir.getAbsolutePath)
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.network.SocketServerConfigs
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.storage.internals.checkpoint.PartitionMetadataFile
import org.mockito.invocation.InvocationOnMock
@ -2616,9 +2616,9 @@ class ReplicaManagerTest {
def testFullLeaderAndIsrStrayPartitions(zkMigrationEnabled: Boolean): Unit = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
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(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")
config = KafkaConfig.fromProps(props)
}
@ -6451,9 +6451,9 @@ class ReplicaManagerTest {
timer = new MockTimer(time),
aliveBrokerIds = Seq(0, 1, 2),
propsModifier = props => {
props.setProperty(KafkaConfig.MigrationEnabledProp, "true")
props.setProperty(KafkaConfig.QuorumVotersProp, "1000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, "true")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "1000@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "CONTROLLER:PLAINTEXT,PLAINTEXT:PLAINTEXT")
},
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.utils.Time
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.Assertions._
import org.junit.jupiter.api.function.Executable
@ -147,7 +147,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
@ValueSource(strings = Array("zk", "kraft"))
def testCleanShutdownAfterFailedStartup(quorum: String): Unit = {
if (isKRaftTest()) {
propsToChangeUponRestart.setProperty(KafkaConfig.InitialBrokerRegistrationTimeoutMsProp, "1000")
propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000")
shutdownBroker()
shutdownKRaftController()
verifyCleanShutdownAfterFailedStartup[CancellationException]

View File

@ -20,6 +20,8 @@ package kafka.server
import kafka.utils.TestUtils
import org.apache.kafka.common.config.ConfigException
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.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig}
import org.apache.zookeeper.KeeperException.NodeExistsException
@ -140,11 +142,11 @@ class ServerStartupTest extends QuorumTestHarness {
@ValueSource(booleans = Array(false, true))
def testDirectoryIdsCreatedOnlyForMigration(migrationEnabled: Boolean): Unit = {
val props = TestUtils.createBrokerConfig(1, zkConnect)
props.setProperty(KafkaConfig.MigrationEnabledProp, migrationEnabled.toString)
props.setProperty(KRaftConfigs.MIGRATION_ENABLED_CONFIG, migrationEnabled.toString)
if (migrationEnabled) {
// Create Controller properties needed when migration is enabled
props.setProperty(KafkaConfig.QuorumVotersProp, "3000@localhost:9093")
props.setProperty(KafkaConfig.ControllerListenerNamesProp, "CONTROLLER")
props.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, "3000@localhost:9093")
props.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG,
"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.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.Test
@ -34,10 +35,10 @@ class ServerTest {
val clusterId = Uuid.randomUuid().toString
val props = new Properties()
props.put(KafkaConfig.ProcessRolesProp, "broker")
props.put(KafkaConfig.NodeIdProp, nodeId.toString)
props.put(KafkaConfig.QuorumVotersProp, s"${nodeId + 1}@localhost:9093")
props.put(KafkaConfig.ControllerListenerNamesProp, "SSL")
props.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker")
props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, s"${nodeId + 1}@localhost:9093")
props.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
val config = KafkaConfig.fromProps(props)
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.common.metadata.UserScramCredentialRecord
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.{Test, Timeout}
import org.junit.jupiter.params.ParameterizedTest
@ -46,10 +47,10 @@ class StorageToolTest {
private def newSelfManagedProperties() = {
val properties = new Properties()
properties.setProperty(ServerLogConfigs.LOG_DIRS_CONFIG, "/tmp/foo,/tmp/bar")
properties.setProperty(KafkaConfig.ProcessRolesProp, "controller")
properties.setProperty(KafkaConfig.NodeIdProp, "2")
properties.setProperty(KafkaConfig.QuorumVotersProp, s"2@localhost:9092")
properties.setProperty(KafkaConfig.ControllerListenerNamesProp, "PLAINTEXT")
properties.setProperty(KRaftConfigs.PROCESS_ROLES_CONFIG, "controller")
properties.setProperty(KRaftConfigs.NODE_ID_CONFIG, "2")
properties.setProperty(QuorumConfig.QUORUM_VOTERS_CONFIG, s"2@localhost:9092")
properties.setProperty(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "PLAINTEXT")
properties
}
@ -62,7 +63,7 @@ class StorageToolTest {
@Test
def testConfigToLogDirectoriesWithMetaLogDir(): Unit = {
val properties = newSelfManagedProperties()
properties.setProperty(KafkaConfig.MetadataLogDirProp, "/tmp/baz")
properties.setProperty(KRaftConfigs.METADATA_LOG_DIR_CONFIG, "/tmp/baz")
val config = new KafkaConfig(properties)
assertEquals(Seq("/tmp/bar", "/tmp/baz", "/tmp/foo"),
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.metadata.properties.MetaProperties
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.QuorumConfig
import org.apache.kafka.server.{ClientMetricsManager, ControllerRequestCompletionHandler}
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.metrics.KafkaYammerMetrics
import org.apache.kafka.server.util.MockTime
@ -333,12 +334,12 @@ object TestUtils extends Logging {
val props = new Properties
props.put(KafkaConfig.UnstableMetadataVersionsEnableProp, "true")
if (zkConnect == null) {
props.setProperty(KafkaConfig.ServerMaxStartupTimeMsProp, TimeUnit.MINUTES.toMillis(10).toString)
props.put(KafkaConfig.NodeIdProp, nodeId.toString)
props.setProperty(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG, TimeUnit.MINUTES.toMillis(10).toString)
props.put(KRaftConfigs.NODE_ID_CONFIG, nodeId.toString)
props.put(KafkaConfig.BrokerIdProp, nodeId.toString)
props.put(SocketServerConfigs.ADVERTISED_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.
map(p => "%s:%s".format(p._1, p._1)).mkString(",") + ",CONTROLLER:PLAINTEXT")
} else {
@ -356,12 +357,12 @@ object TestUtils extends Logging {
props.put(ServerLogConfigs.LOG_DIR_CONFIG, tempDir().getAbsolutePath)
}
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
// tests use random port assignment, so the controller ports are not known ahead of
// time. Therefore, we ignore controller.quorum.voters and use
// controllerQuorumVotersFuture instead.
props.put(KafkaConfig.QuorumVotersProp, "1000@localhost:0")
props.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "1000@localhost:0")
} else {
props.put(ZkConfigs.ZK_CONNECT_CONFIG, zkConnect)
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.KafkaApis;
import kafka.server.KafkaConfig;
import kafka.server.KafkaConfig$;
import kafka.server.MetadataCache;
import kafka.server.QuotaFactory;
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.MetadataImage;
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.MetadataVersion;
import org.apache.kafka.server.config.KRaftConfigs;
import org.mockito.Mockito;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@ -174,10 +176,10 @@ public class KRaftMetadataRequestBenchmark {
private KafkaApis createKafkaApis() {
Properties kafkaProps = new Properties();
kafkaProps.put(KafkaConfig$.MODULE$.NodeIdProp(), brokerId + "");
kafkaProps.put(KafkaConfig$.MODULE$.ProcessRolesProp(), "broker");
kafkaProps.put(KafkaConfig$.MODULE$.QuorumVotersProp(), "9000@foo:8092");
kafkaProps.put(KafkaConfig$.MODULE$.ControllerListenerNamesProp(), "CONTROLLER");
kafkaProps.put(KRaftConfigs.NODE_ID_CONFIG, brokerId + "");
kafkaProps.put(KRaftConfigs.PROCESS_ROLES_CONFIG, "broker");
kafkaProps.put(QuorumConfig.QUORUM_VOTERS_CONFIG, "9000@foo:8092");
kafkaProps.put(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER");
KafkaConfig config = new KafkaConfig(kafkaProps);
return new KafkaApisBuilder().
setRequestChannel(requestChannel).

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.raft.QuorumConfig;
import java.util.List;
import java.util.concurrent.TimeUnit;
public class Defaults {
/** ********* General Configuration *********/
@ -33,13 +32,6 @@ public class Defaults {
public static final int BACKGROUND_THREADS = 10;
public static final int QUEUED_MAX_REQUESTS = 500;
public static final int QUEUED_MAX_REQUEST_BYTES = -1;
public static final int INITIAL_BROKER_REGISTRATION_TIMEOUT_MS = 60000;
public static final int BROKER_HEARTBEAT_INTERVAL_MS = 2000;
public static final int BROKER_SESSION_TIMEOUT_MS = 9000;
public static final int METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES = 20 * 1024 * 1024;
public static final long METADATA_SNAPSHOT_MAX_INTERVAL_MS = TimeUnit.HOURS.toMillis(1);
public static final int METADATA_MAX_IDLE_INTERVAL_MS = 500;
public static final int METADATA_MAX_RETENTION_BYTES = 100 * 1024 * 1024;
public static final boolean DELETE_TOPIC_ENABLE = true;
public static final int REQUEST_TIMEOUT_MS = 30000;
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";
}