mirror of https://github.com/apache/kafka.git
KAFKA-15853 Move KRAFT configs out of KafkaConfig (#15775)
Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
ba6a73776b
commit
d88c15fc3e
|
@ -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')
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.")
|
||||
}
|
||||
|
||||
|
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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`.
|
||||
*
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 ++
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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.",
|
||||
|
|
|
@ -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) =
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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")
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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";
|
||||
}
|
Loading…
Reference in New Issue