diff --git a/build.gradle b/build.gradle
index f0e1e8ec08540..b17da64326142 100644
--- a/build.gradle
+++ b/build.gradle
@@ -865,6 +865,8 @@ project(':server') {
implementation project(':server-common')
implementation project(':storage')
implementation project(':group-coordinator')
+ implementation project(':group-coordinator:group-coordinator-api')
+ implementation libs.commonsValidator
implementation project(':transaction-coordinator')
implementation project(':raft')
implementation libs.metrics
diff --git a/checkstyle/import-control-server.xml b/checkstyle/import-control-server.xml
index 27b7bcda8dfde..e7a72f9136a28 100644
--- a/checkstyle/import-control-server.xml
+++ b/checkstyle/import-control-server.xml
@@ -89,6 +89,9 @@
+
+
+
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index a48d6cbd7a305..e3daa60f8b3f6 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -1655,6 +1655,16 @@ public static void require(boolean requirement) {
throw new IllegalArgumentException("requirement failed");
}
+ /**
+ * Checks requirement. Throw {@link IllegalArgumentException} if {@code requirement} failed.
+ * @param requirement Requirement to check.
+ * @param message String to include in the failure message
+ */
+ public static void require(boolean requirement, String message) {
+ if (!requirement)
+ throw new IllegalArgumentException("requirement failed: " + message);
+ }
+
/**
* Merge multiple {@link ConfigDef} into one
* @param configDefs List of {@link ConfigDef}
@@ -1664,6 +1674,7 @@ public static ConfigDef mergeConfigs(List configDefs) {
configDefs.forEach(configDef -> configDef.configKeys().values().forEach(all::define));
return all;
}
+
/**
* A runnable that can throw checked exception.
*/
diff --git a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java
index a62987be2036e..c2e575da223e5 100644
--- a/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java
+++ b/core/src/main/java/kafka/server/MetadataVersionConfigValidator.java
@@ -60,7 +60,7 @@ public void onMetadataUpdate(
@SuppressWarnings("ThrowableNotThrown")
private void onMetadataVersionChanged(MetadataVersion metadataVersion) {
try {
- this.config.validateWithMetadataVersion(metadataVersion);
+ this.config.validator().validateWithMetadataVersion(metadataVersion);
} catch (Throwable t) {
this.faultHandler.handleFault(
"Broker configuration does not support the cluster MetadataVersion", t);
diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala
index e5835201fa356..bf08ea8586129 100755
--- a/core/src/main/scala/kafka/cluster/Broker.scala
+++ b/core/src/main/scala/kafka/cluster/Broker.scala
@@ -116,6 +116,6 @@ case class Broker(id: Int, endPoints: Seq[EndPoint], rack: Option[String], featu
val interBrokerEndpoint: Endpoint = endPoint(config.interBrokerListenerName).toJava
val brokerEndpoints: util.List[Endpoint] = endPoints.toList.map(_.toJava).asJava
Broker.ServerInfo(clusterResource, id, brokerEndpoints, interBrokerEndpoint,
- config.earlyStartListeners.map(_.value()).asJava)
+ config.earlyStartListeners().asScala.map(_.value()).asJava)
}
}
diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
index 793b39538e758..2dd68b0bc489c 100755
--- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala
@@ -117,8 +117,8 @@ class ControllerChannelManager(controllerEpoch: () => Int,
private def addNewBroker(broker: Broker): Unit = {
val messageQueue = new LinkedBlockingQueue[QueueItem]
debug(s"Controller ${config.brokerId} trying to connect to broker ${broker.id}")
- val controllerToBrokerListenerName = config.controlPlaneListenerName.getOrElse(config.interBrokerListenerName)
- val controllerToBrokerSecurityProtocol = config.controlPlaneSecurityProtocol.getOrElse(config.interBrokerSecurityProtocol)
+ val controllerToBrokerListenerName = config.controlPlaneListenerName.orElse(config.interBrokerListenerName)
+ val controllerToBrokerSecurityProtocol = config.controlPlaneSecurityProtocol.orElse(config.interBrokerSecurityProtocol)
val brokerNode = broker.node(controllerToBrokerListenerName)
val logContext = new LogContext(s"[Controller id=${config.brokerId}, targetBrokerId=${brokerNode.idString}] ")
val (networkClient, reconfigurableChannelBuilder) = {
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index 8908aadf45930..be103bccdbc61 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -1567,7 +1567,7 @@ object LogManager {
val cleanerConfig = LogCleaner.cleanerConfig(config)
- new LogManager(logDirs = config.logDirs.map(new File(_).getAbsoluteFile),
+ new LogManager(logDirs = config.logDirs.asScala.map(new File(_).getAbsoluteFile),
initialOfflineDirs = initialOfflineDirs.map(new File(_).getAbsoluteFile),
configRepository = configRepository,
initialDefaultConfig = defaultLogConfig,
diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala
index 42d1e6585e6cf..3c1e5a92be8ac 100644
--- a/core/src/main/scala/kafka/network/SocketServer.scala
+++ b/core/src/main/scala/kafka/network/SocketServer.scala
@@ -47,13 +47,14 @@ import org.apache.kafka.common.utils.{KafkaThread, LogContext, Time, Utils}
import org.apache.kafka.common.{Endpoint, KafkaException, MetricName, Reconfigurable}
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.CredentialProvider
-import org.apache.kafka.server.config.{ServerConfigs, QuotaConfigs}
+import org.apache.kafka.server.config.{QuotaConfigs, ServerConfigs}
import org.apache.kafka.server.metrics.KafkaMetricsGroup
import org.apache.kafka.server.util.FutureUtils
import org.slf4j.event.Level
import scala.collection._
import scala.collection.mutable.ArrayBuffer
+import scala.compat.java8.OptionConverters.RichOptionalGeneric
import scala.jdk.CollectionConverters._
import scala.util.control.ControlThrowable
@@ -102,7 +103,7 @@ class SocketServer(val config: KafkaConfig,
val dataPlaneRequestChannel = new RequestChannel(maxQueuedRequests, DataPlaneAcceptor.MetricPrefix, time, apiVersionManager.newRequestMetrics)
// control-plane
private[network] var controlPlaneAcceptorOpt: Option[ControlPlaneAcceptor] = None
- val controlPlaneRequestChannelOpt: Option[RequestChannel] = config.controlPlaneListenerName.map(_ =>
+ val controlPlaneRequestChannelOpt: Option[RequestChannel] = config.controlPlaneListenerName.asScala.map(_ =>
new RequestChannel(20, ControlPlaneAcceptor.MetricPrefix, time, apiVersionManager.newRequestMetrics))
private[this] val nextProcessorId: AtomicInteger = new AtomicInteger(0)
@@ -171,10 +172,10 @@ class SocketServer(val config: KafkaConfig,
// structures. It does not start the acceptors and processors or their associated JVM
// threads.
if (apiVersionManager.listenerType.equals(ListenerType.CONTROLLER)) {
- config.controllerListeners.foreach(createDataPlaneAcceptorAndProcessors)
+ config.controllerListeners.forEach(l => createDataPlaneAcceptorAndProcessors(EndPoint.fromJava(l)))
} else {
- config.controlPlaneListener.foreach(createControlPlaneAcceptorAndProcessor)
- config.dataPlaneListeners.foreach(createDataPlaneAcceptorAndProcessors)
+ config.controlPlaneListener.map(l => createControlPlaneAcceptorAndProcessor(EndPoint.fromJava(l)))
+ config.dataPlaneListeners.forEach(l => createDataPlaneAcceptorAndProcessors(EndPoint.fromJava(l)))
}
// Processors are now created by each Acceptor. However to preserve compatibility, we need to number the processors
@@ -266,7 +267,10 @@ class SocketServer(val config: KafkaConfig,
info(s"Created control-plane acceptor and processor for endpoint : ${endpoint.listenerName}")
}
- private def endpoints = config.listeners.map(l => l.listenerName -> l).toMap
+ private def endpoints = config.listeners.asScala.map(l => {
+ val endpoint = EndPoint.fromJava(l)
+ endpoint.listenerName -> endpoint
+ }).toMap
protected def createDataPlaneAcceptor(endPoint: EndPoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel): DataPlaneAcceptor = {
new DataPlaneAcceptor(this, endPoint, config, nodeId, connectionQuotas, time, isPrivilegedListener, requestChannel, metrics, credentialProvider, logContext, memoryPool, apiVersionManager)
@@ -327,14 +331,15 @@ class SocketServer(val config: KafkaConfig,
/**
* This method is called to dynamically add listeners.
*/
- def addListeners(listenersAdded: Seq[EndPoint]): Unit = synchronized {
+ def addListeners(listenersAdded: Seq[Endpoint]): Unit = synchronized {
if (stopped) {
throw new RuntimeException("can't add new listeners: SocketServer is stopped.")
}
info(s"Adding data-plane listeners for endpoints $listenersAdded")
- listenersAdded.foreach { endpoint =>
+ listenersAdded.foreach { l =>
+ val endpoint = EndPoint.fromJava(l)
createDataPlaneAcceptorAndProcessors(endpoint)
- val acceptor = dataPlaneAcceptors.get(endpoint)
+ val acceptor = dataPlaneAcceptors.get(l)
// There is no authorizer future for this new listener endpoint. So start the
// listener once all authorizer futures are complete.
allAuthorizerFuturesComplete.whenComplete((_, e) => {
@@ -371,8 +376,8 @@ class SocketServer(val config: KafkaConfig,
info(s"Updating maxConnectionsPerIp: $maxConnectionsPerIp")
connectionQuotas.updateMaxConnectionsPerIp(maxConnectionsPerIp)
}
- val maxConnectionsPerIpOverrides = newConfig.maxConnectionsPerIpOverrides
- if (maxConnectionsPerIpOverrides != oldConfig.maxConnectionsPerIpOverrides) {
+ val maxConnectionsPerIpOverrides = newConfig.maxConnectionsPerIpOverrides.asScala.view.mapValues(_.toInt).toMap
+ if (maxConnectionsPerIpOverrides != oldConfig.maxConnectionsPerIpOverrides.asScala.view.mapValues(_.toInt).toMap) {
info(s"Updating maxConnectionsPerIpOverrides: ${maxConnectionsPerIpOverrides.map { case (k, v) => s"$k=$v" }.mkString(",")}")
connectionQuotas.updateMaxConnectionsPerIpOverride(maxConnectionsPerIpOverrides)
}
@@ -1435,7 +1440,7 @@ object ConnectionQuotas {
class ConnectionQuotas(config: KafkaConfig, time: Time, metrics: Metrics) extends Logging with AutoCloseable {
@volatile private var defaultMaxConnectionsPerIp: Int = config.maxConnectionsPerIp
- @volatile private var maxConnectionsPerIpOverrides = config.maxConnectionsPerIpOverrides.map { case (host, count) => (InetAddress.getByName(host), count) }
+ @volatile private var maxConnectionsPerIpOverrides = config.maxConnectionsPerIpOverrides.asScala.map { case (host, count) => (InetAddress.getByName(host), count.toInt) }.toMap
@volatile private var brokerMaxConnections = config.maxConnections
private val interBrokerListenerName = config.interBrokerListenerName
private val counts = mutable.Map[InetAddress, Int]()
@@ -1473,7 +1478,7 @@ class ConnectionQuotas(config: KafkaConfig, time: Time, metrics: Metrics) extend
}
private[network] def updateMaxConnectionsPerIpOverride(overrideQuotas: Map[String, Int]): Unit = {
- maxConnectionsPerIpOverrides = overrideQuotas.map { case (host, count) => (InetAddress.getByName(host), count) }
+ maxConnectionsPerIpOverrides = overrideQuotas.map { case (host, count) => (InetAddress.getByName(host), count) }.toMap
}
private[network] def updateBrokerMaxConnections(maxConnections: Int): Unit = {
diff --git a/core/src/main/scala/kafka/raft/RaftManager.scala b/core/src/main/scala/kafka/raft/RaftManager.scala
index 65ef855640cd6..5f8e8c33d77c5 100644
--- a/core/src/main/scala/kafka/raft/RaftManager.scala
+++ b/core/src/main/scala/kafka/raft/RaftManager.scala
@@ -79,7 +79,7 @@ object KafkaRaftManager {
*/
private def hasDifferentLogDir(config: KafkaConfig): Boolean = {
!config
- .logDirs
+ .logDirs.asScala
.map(Paths.get(_).toAbsolutePath)
.contains(Paths.get(config.metadataLogDir).toAbsolutePath)
}
@@ -95,7 +95,7 @@ object KafkaRaftManager {
*/
def maybeDeleteMetadataLogDir(config: KafkaConfig): Unit = {
// These constraints are enforced in KafkaServer, but repeating them here to guard against future callers
- if (config.processRoles.nonEmpty) {
+ if (!config.processRoles.isEmpty) {
throw new RuntimeException("Not deleting metadata log dir since this node is in KRaft mode.")
} else if (!config.migrationEnabled) {
throw new RuntimeException("Not deleting metadata log dir since migrations are not enabled.")
@@ -171,7 +171,7 @@ class KafkaRaftManager[T](
val differentMetadataLogDir = KafkaRaftManager.hasDifferentLogDir(config)
// Or this node is only a controller
- val isOnlyController = config.processRoles == Set(ProcessRole.ControllerRole)
+ val isOnlyController = config.processRoles == Utils.mkSet(ProcessRole.ControllerRole)
if (differentMetadataLogDir || isOnlyController) {
Some(KafkaRaftManager.lockDataDir(new File(config.metadataLogDir)))
@@ -260,8 +260,8 @@ class KafkaRaftManager[T](
}
private def buildNetworkClient(): (ListenerName, NetworkClient) = {
- val controllerListenerName = new ListenerName(config.controllerListenerNames.head)
- val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(
+ val controllerListenerName = new ListenerName(config.controllerListenerNames.asScala.head)
+ val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.asScala.getOrElse(
controllerListenerName,
SecurityProtocol.forName(controllerListenerName.value())
)
diff --git a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
index 51bc16fb09d17..82f1311cbd004 100644
--- a/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
+++ b/core/src/main/scala/kafka/server/BrokerLifecycleManager.scala
@@ -398,7 +398,7 @@ class BrokerLifecycleManager(
setFeatures(features).
setIncarnationId(incarnationId).
setListeners(_advertisedListeners).
- setRack(rack.orNull).
+ setRack(rack.orElse(null)).
setPreviousBrokerEpoch(previousBrokerEpoch.orElse(-1L)).
setLogDirs(sortedLogDirs)
if (isDebugEnabled) {
diff --git a/core/src/main/scala/kafka/server/BrokerServer.scala b/core/src/main/scala/kafka/server/BrokerServer.scala
index a03128ab9f52b..e30d977cf250f 100644
--- a/core/src/main/scala/kafka/server/BrokerServer.scala
+++ b/core/src/main/scala/kafka/server/BrokerServer.scala
@@ -36,7 +36,7 @@ import org.apache.kafka.common.security.token.delegation.internals.DelegationTok
import org.apache.kafka.common.utils.{LogContext, Time}
import org.apache.kafka.common.{ClusterResource, TopicPartition, Uuid}
import org.apache.kafka.coordinator.group.metrics.{GroupCoordinatorMetrics, GroupCoordinatorRuntimeMetrics}
-import org.apache.kafka.coordinator.group.{CoordinatorRecord, GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService, CoordinatorRecordSerde}
+import org.apache.kafka.coordinator.group.{CoordinatorRecord, CoordinatorRecordSerde, GroupCoordinator, GroupCoordinatorConfig, GroupCoordinatorService}
import org.apache.kafka.image.publisher.{BrokerRegistrationTracker, MetadataPublisher}
import org.apache.kafka.metadata.{BrokerState, ListenerInfo, VersionRange}
import org.apache.kafka.security.CredentialProvider
@@ -56,9 +56,10 @@ import java.util
import java.util.Optional
import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.{Condition, ReentrantLock}
-import java.util.concurrent.{CompletableFuture, ExecutionException, TimeoutException, TimeUnit}
+import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit, TimeoutException}
+import java.util.stream.Collectors
import scala.collection.Map
-import scala.compat.java8.OptionConverters.RichOptionForJava8
+import scala.compat.java8.OptionConverters.{RichOptionForJava8, RichOptionalGeneric}
import scala.jdk.CollectionConverters._
@@ -255,7 +256,7 @@ class BrokerServer(
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
val listenerInfo = ListenerInfo.create(Optional.of(config.interBrokerListenerName.value()),
- config.effectiveAdvertisedListeners.map(_.toJava).asJava).
+ config.effectiveAdvertisedListeners).
withWildcardHostnamesResolved().
withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
@@ -386,7 +387,7 @@ class BrokerServer(
})
// Create and initialize an authorizer if one is configured.
- authorizer = config.createNewAuthorizer()
+ authorizer = config.createNewAuthorizer().asScala
authorizer.foreach(_.configure(config.originals))
// The FetchSessionCache is divided into config.numIoThreads shards, each responsible
@@ -530,13 +531,17 @@ class BrokerServer(
// authorizer future is completed.
val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder()
+ val earlyStartListeners: util.Set[String] = config.earlyStartListeners.stream()
+ .map(_.value())
+ .collect(Collectors.toSet())
+
builder.build(authorizer.asJava,
new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId),
config.nodeId,
listenerInfo.listeners().values(),
listenerInfo.firstListener(),
- config.earlyStartListeners.map(_.value()).asJava))
+ earlyStartListeners))
}
val authorizerFutures = endpointReadyFutures.futures().asScala.toMap
val enableRequestProcessingFuture = socketServer.enableRequestProcessing(authorizerFutures)
@@ -621,7 +626,7 @@ class BrokerServer(
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
- Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
+ Some(new RemoteLogManager(config, config.brokerId, config.logDirs.asScala.head, clusterId, time,
(tp: TopicPartition) => logManager.getLog(tp).asJava,
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
logManager.getLog(tp).foreach { log =>
diff --git a/core/src/main/scala/kafka/server/ControllerServer.scala b/core/src/main/scala/kafka/server/ControllerServer.scala
index 7c1f694b08030..d5a4bcf56ae87 100644
--- a/core/src/main/scala/kafka/server/ControllerServer.scala
+++ b/core/src/main/scala/kafka/server/ControllerServer.scala
@@ -56,6 +56,7 @@ import java.util
import java.util.{Optional, OptionalLong}
import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.{CompletableFuture, TimeUnit}
+import java.util.stream.Collectors
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
@@ -159,7 +160,7 @@ class ControllerServer(
metricsGroup.newGauge("linux-disk-write-bytes", () => linuxIoMetricsCollector.writeBytes())
}
- authorizer = config.createNewAuthorizer()
+ authorizer = config.createNewAuthorizer().asScala
authorizer.foreach(_.configure(config.originals))
metadataCache = MetadataCache.kRaftMetadataCache(config.nodeId)
@@ -187,20 +188,24 @@ class ControllerServer(
credentialProvider,
apiVersionManager)
- val listenerInfo = ListenerInfo.create(config.controllerListeners.map(_.toJava).asJava).
+ val listenerInfo = ListenerInfo.create(config.controllerListeners).
withWildcardHostnamesResolved().
withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port())
val endpointReadyFutures = {
val builder = new EndpointReadyFutures.Builder()
+ val earlyStartListeners: util.Set[String] = config.earlyStartListeners.stream()
+ .map(_.value())
+ .collect(Collectors.toSet())
+
builder.build(authorizer.asJava,
new KafkaAuthorizerServerInfo(
new ClusterResource(clusterId),
config.nodeId,
listenerInfo.listeners().values(),
listenerInfo.firstListener(),
- config.earlyStartListeners.map(_.value()).asJava))
+ earlyStartListeners))
}
sharedServer.startForController()
@@ -234,7 +239,7 @@ class ControllerServer(
OptionalLong.empty()
}
- val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.fold(OptionalLong.empty)(OptionalLong.of)
+ val maxIdleIntervalNs = config.metadataMaxIdleIntervalNs.asScala.fold(OptionalLong.empty)(interval => OptionalLong.of(interval))
quorumControllerMetrics = new QuorumControllerMetrics(Optional.of(KafkaYammerMetrics.defaultRegistry), time, config.migrationEnabled)
@@ -278,7 +283,7 @@ class ControllerServer(
if (config.migrationEnabled) {
val zkClient = KafkaZkClient.createZkClient("KRaft Migration", time, config, KafkaServer.zkClientConfigFromKafkaConfig(config))
- val zkConfigEncoder = config.passwordEncoderSecret match {
+ val zkConfigEncoder = config.passwordEncoderSecret.asScala match {
case Some(secret) => PasswordEncoder.encrypting(secret,
config.passwordEncoderKeyFactoryAlgorithm,
config.passwordEncoderCipherAlgorithm,
diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
index 96566826b0432..88c7b193022a7 100755
--- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
+++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala
@@ -17,37 +17,38 @@
package kafka.server
-import java.util
-import java.util.{Collections, Properties}
-import java.util.concurrent.CopyOnWriteArrayList
-import java.util.concurrent.locks.ReentrantReadWriteLock
import kafka.cluster.EndPoint
import kafka.log.{LogCleaner, LogManager}
import kafka.network.{DataPlaneAcceptor, SocketServer}
import kafka.server.DynamicBrokerConfig._
-import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._
+import kafka.utils.{CoreUtils, Logging}
import kafka.zk.{AdminZkClient, KafkaZkClient}
-import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
-import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, SaslConfigs, SslConfigs}
-import org.apache.kafka.common.metrics.{JmxReporter, Metrics, MetricsReporter}
import org.apache.kafka.common.config.types.Password
+import org.apache.kafka.common.config._
+import org.apache.kafka.common.metrics.{JmxReporter, Metrics, MetricsReporter}
import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable}
import org.apache.kafka.common.security.authenticator.LoginManager
import org.apache.kafka.common.utils.{ConfigUtils, Utils}
+import org.apache.kafka.common.{Endpoint, Reconfigurable}
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.security.PasswordEncoder
import org.apache.kafka.server.ProcessRole
-import org.apache.kafka.server.config.{ConfigType, ServerConfigs, ReplicationConfigs, ServerLogConfigs, ServerTopicConfigSynonyms, ZooKeeperInternals}
+import org.apache.kafka.server.config._
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
import org.apache.kafka.server.metrics.{ClientMetricsReceiverPlugin, MetricConfigs}
import org.apache.kafka.server.telemetry.ClientTelemetry
import org.apache.kafka.storage.internals.log.{LogConfig, ProducerStateManagerConfig}
+import java.util
+import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.locks.ReentrantReadWriteLock
+import java.util.{Collections, Properties}
import scala.annotation.nowarn
import scala.collection._
+import scala.compat.java8.OptionConverters.RichOptionalGeneric
import scala.jdk.CollectionConverters._
/**
@@ -216,7 +217,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
private var metricsReceiverPluginOpt: Option[ClientMetricsReceiverPlugin] = _
private var currentConfig: KafkaConfig = _
private val dynamicConfigPasswordEncoder = if (kafkaConfig.processRoles.isEmpty) {
- maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret)
+ maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderSecret.asScala)
} else {
Some(PasswordEncoder.NOOP)
}
@@ -448,7 +449,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
private def maybeReEncodePasswords(persistentProps: Properties, adminZkClient: AdminZkClient): Properties = {
val props = persistentProps.clone().asInstanceOf[Properties]
if (props.asScala.keySet.exists(isPasswordConfig)) {
- maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderOldSecret).foreach { passwordDecoder =>
+ maybeCreatePasswordEncoder(kafkaConfig.passwordEncoderOldSecret.asScala).foreach { passwordDecoder =>
persistentProps.asScala.forKeyValue { (configName, value) =>
if (isPasswordConfig(configName) && value != null) {
val decoded = try {
@@ -1073,7 +1074,7 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
val oldListeners = listenersToMap(oldConfig.listeners)
if (!newAdvertisedListeners.keySet.subsetOf(newListeners.keySet))
throw new ConfigException(s"Advertised listeners '$newAdvertisedListeners' must be a subset of listeners '$newListeners'")
- if (!newListeners.keySet.subsetOf(newConfig.effectiveListenerSecurityProtocolMap.keySet))
+ if (!newListeners.keySet.subsetOf(newConfig.effectiveListenerSecurityProtocolMap.asScala.keySet))
throw new ConfigException(s"Listeners '$newListeners' must be subset of listener map '${newConfig.effectiveListenerSecurityProtocolMap}'")
newListeners.keySet.intersect(oldListeners.keySet).foreach { listenerName =>
def immutableListenerConfigs(kafkaConfig: KafkaConfig, prefix: String): Map[String, AnyRef] = {
@@ -1085,7 +1086,7 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
if (immutableListenerConfigs(newConfig, listenerName.configPrefix) != immutableListenerConfigs(oldConfig, listenerName.configPrefix))
throw new ConfigException(s"Configs cannot be updated dynamically for existing listener $listenerName, " +
"restart broker or create a new listener for update")
- if (oldConfig.effectiveListenerSecurityProtocolMap(listenerName) != newConfig.effectiveListenerSecurityProtocolMap(listenerName))
+ if (oldConfig.effectiveListenerSecurityProtocolMap.get(listenerName) != newConfig.effectiveListenerSecurityProtocolMap.get(listenerName))
throw new ConfigException(s"Security protocol cannot be updated for existing listener $listenerName")
}
if (!newAdvertisedListeners.contains(newConfig.interBrokerListenerName))
@@ -1104,8 +1105,8 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
val newListenerMap = listenersToMap(newListeners)
val oldListeners = oldConfig.listeners
val oldListenerMap = listenersToMap(oldListeners)
- val listenersRemoved = oldListeners.filterNot(e => newListenerMap.contains(e.listenerName))
- val listenersAdded = newListeners.filterNot(e => oldListenerMap.contains(e.listenerName))
+ val listenersRemoved = oldListeners.asScala.map(EndPoint.fromJava).filterNot(e => newListenerMap.contains(e.listenerName)).toSeq
+ val listenersAdded = newListeners.asScala.filterNot(e => oldListenerMap.contains(EndPoint.fromJava(e).listenerName)).toSeq
if (listenersRemoved.nonEmpty || listenersAdded.nonEmpty) {
LoginManager.closeAll() // Clear SASL login cache to force re-login
if (listenersRemoved.nonEmpty) server.socketServer.removeListeners(listenersRemoved)
@@ -1121,6 +1122,8 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
}
}
+ private def listenersToMap(listeners: util.List[Endpoint]): Map[ListenerName, EndPoint] =
+ listenersToMap(listeners.asScala.map(EndPoint.fromJava))
private def listenersToMap(listeners: Seq[EndPoint]): Map[ListenerName, EndPoint] =
listeners.map(e => (e.listenerName, e)).toMap
diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala
index f2cddda700fca..48e9d127cc31c 100755
--- a/core/src/main/scala/kafka/server/KafkaConfig.scala
+++ b/core/src/main/scala/kafka/server/KafkaConfig.scala
@@ -17,47 +17,24 @@
package kafka.server
-import java.util
-import java.util.concurrent.TimeUnit
-import java.util.Properties
-import kafka.cluster.EndPoint
-import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._
+import kafka.utils.Logging
import org.apache.kafka.common.Reconfigurable
-import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, SaslConfigs, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.ConfigKey
-import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.types.Password
-import org.apache.kafka.common.network.ListenerName
-import org.apache.kafka.common.record.{CompressionType, TimestampType}
-import org.apache.kafka.common.security.auth.KafkaPrincipalSerde
-import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource}
import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
-import org.apache.kafka.coordinator.group.Group.GroupType
-import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
-import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor
-import org.apache.kafka.coordinator.transaction.{TransactionLogConfigs, TransactionStateManagerConfigs}
-import org.apache.kafka.network.SocketServerConfigs
-import org.apache.kafka.raft.QuorumConfig
-import org.apache.kafka.security.authorizer.AuthorizerUtils
-import org.apache.kafka.security.PasswordEncoderConfigs
import org.apache.kafka.server.ProcessRole
-import org.apache.kafka.server.authorizer.Authorizer
import org.apache.kafka.server.common.MetadataVersion
-import org.apache.kafka.server.common.MetadataVersion._
-import org.apache.kafka.server.config.{AbstractKafkaConfig, DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfigs, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ShareGroupConfigs, ZkConfigs}
-import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
-import org.apache.kafka.server.metrics.MetricConfigs
-import org.apache.kafka.server.util.Csv
-import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
-import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
+import org.apache.kafka.server.config._
+import org.apache.kafka.storage.internals.log.LogConfig
import org.apache.zookeeper.client.ZKClientConfig
-import scala.annotation.nowarn
+import java.util
+import java.util.{Collections, Properties}
+import scala.collection.{Map, Seq}
import scala.compat.java8.OptionConverters._
import scala.jdk.CollectionConverters._
-import scala.collection.{Map, Seq}
object KafkaConfig {
@@ -90,6 +67,7 @@ object KafkaConfig {
zooKeeperClientProperty(zkClientConfig, ZkConfigs.ZK_SSL_KEY_STORE_LOCATION_CONFIG).isDefined
}
+ /** ********* Remote Log Management Configuration *********/
val configDef = AbstractKafkaConfig.CONFIG_DEF
def configNames: Seq[String] = configDef.names.asScala.toBuffer.sorted
@@ -189,8 +167,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
this(doLog, KafkaConfig.populateSynonyms(props), dynamicConfigOverride)
// Cache the current config to avoid acquiring read lock to access from dynamicConfig
- @volatile private var currentConfig = this
- val processRoles: Set[ProcessRole] = parseProcessRoles()
private[server] val dynamicConfig = dynamicConfigOverride.getOrElse(new DynamicBrokerConfig(this))
private[server] def updateCurrentConfig(newConfig: KafkaConfig): Unit = {
@@ -202,28 +178,7 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
// We make it part of each instance rather than the object to facilitate testing.
private val zkClientConfigViaSystemProperties = new ZKClientConfig()
- override def originals: util.Map[String, AnyRef] =
- if (this eq currentConfig) super.originals else currentConfig.originals
- override def values: util.Map[String, _] =
- if (this eq currentConfig) super.values else currentConfig.values
- override def nonInternalValues: util.Map[String, _] =
- if (this eq currentConfig) super.nonInternalValues else currentConfig.nonInternalValues
- override def originalsStrings: util.Map[String, String] =
- if (this eq currentConfig) super.originalsStrings else currentConfig.originalsStrings
- override def originalsWithPrefix(prefix: String): util.Map[String, AnyRef] =
- if (this eq currentConfig) super.originalsWithPrefix(prefix) else currentConfig.originalsWithPrefix(prefix)
- override def valuesWithPrefixOverride(prefix: String): util.Map[String, AnyRef] =
- if (this eq currentConfig) super.valuesWithPrefixOverride(prefix) else currentConfig.valuesWithPrefixOverride(prefix)
- override def get(key: String): AnyRef =
- if (this eq currentConfig) super.get(key) else currentConfig.get(key)
-
- // During dynamic update, we use the values from this config, these are only used in DynamicBrokerConfig
- private[server] def originalsFromThisConfig: util.Map[String, AnyRef] = super.originals
- private[server] def valuesFromThisConfig: util.Map[String, _] = super.values
- def valuesFromThisConfigWithPrefixOverride(prefix: String): util.Map[String, AnyRef] =
- super.valuesWithPrefixOverride(prefix)
-
- /** ********* Zookeeper Configuration ***********/
+ /** ********* Zookeeper Configuration ********** */
val zkConnect: String = getString(ZkConfigs.ZK_CONNECT_CONFIG)
val zkSessionTimeoutMs: Int = getInt(ZkConfigs.ZK_SESSION_TIMEOUT_MS_CONFIG)
val zkConnectionTimeoutMs: Int =
@@ -231,9 +186,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
val zkEnableSecureAcls: Boolean = getBoolean(ZkConfigs.ZK_ENABLE_SECURE_ACLS_CONFIG)
val zkMaxInFlightRequests: Int = getInt(ZkConfigs.ZK_MAX_IN_FLIGHT_REQUESTS_CONFIG)
- private val _remoteLogManagerConfig = new RemoteLogManagerConfig(props)
- def remoteLogManagerConfig = _remoteLogManagerConfig
-
private def zkBooleanConfigOrSystemPropertyWithDefaultValue(propKey: String): Boolean = {
// Use the system property if it exists and the Kafka config value was defaulted rather than actually provided
// Need to translate any system property value from true/false (String) to true/false (Boolean)
@@ -305,426 +257,6 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
val ZkSslCrlEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_CRL_ENABLE_CONFIG)
val ZkSslOcspEnable = zkBooleanConfigOrSystemPropertyWithDefaultValue(ZkConfigs.ZK_SSL_OCSP_ENABLE_CONFIG)
- /** ********* General Configuration ***********/
- val brokerIdGenerationEnable: Boolean = getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG)
- val maxReservedBrokerId: Int = getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG)
- var brokerId: Int = getInt(ServerConfigs.BROKER_ID_CONFIG)
- 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(KRaftConfigs.MIGRATION_ENABLED_CONFIG)
- val migrationMetadataMinBatchSize: Int = getInt(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG)
-
- val elrEnabled: Boolean = getBoolean(KRaftConfigs.ELR_ENABLED_CONFIG)
-
- private def parseProcessRoles(): Set[ProcessRole] = {
- 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'" +
- " (only 'broker' and 'controller' are allowed roles)")
- }
-
- val distinctRoles: Set[ProcessRole] = roles.toSet
-
- if (distinctRoles.size != roles.size) {
- throw new ConfigException(s"Duplicate role names found in `${KRaftConfigs.PROCESS_ROLES_CONFIG}`: $roles")
- }
-
- distinctRoles
- }
-
- def isKRaftCombinedMode: Boolean = {
- processRoles == Set(ProcessRole.BrokerRole, ProcessRole.ControllerRole)
- }
-
- def metadataLogDir: String = {
- Option(getString(KRaftConfigs.METADATA_LOG_DIR_CONFIG)) match {
- case Some(dir) => dir
- case None => logDirs.head
- }
- }
-
- 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)
- def metadataNodeIDConfig = getInt(KRaftConfigs.NODE_ID_CONFIG)
- def metadataLogSegmentMinBytes = getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG)
- val serverMaxStartupTimeMs = getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG)
-
- def numNetworkThreads = getInt(ServerConfigs.NUM_NETWORK_THREADS_CONFIG)
- def backgroundThreads = getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG)
- val queuedMaxRequests = getInt(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG)
- val queuedMaxBytes = getLong(ServerConfigs.QUEUED_MAX_BYTES_CONFIG)
- def numIoThreads = getInt(ServerConfigs.NUM_IO_THREADS_CONFIG)
- def messageMaxBytes = getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG)
- val requestTimeoutMs = getInt(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG)
- val connectionSetupTimeoutMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG)
- val connectionSetupTimeoutMaxMs = getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG)
-
- def getNumReplicaAlterLogDirsThreads: Int = {
- val numThreads: Integer = Option(getInt(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).getOrElse(logDirs.size)
- numThreads
- }
-
- /************* Metadata Configuration ***********/
- 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(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG).toLong, TimeUnit.MILLISECONDS)
- if (value > 0) Some(value) else None
- }
-
- /************* Authorizer Configuration ***********/
- def createNewAuthorizer(): Option[Authorizer] = {
- val className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG)
- if (className == null || className.isEmpty)
- None
- else {
- Some(AuthorizerUtils.createAuthorizer(className))
- }
- }
-
- val earlyStartListeners: Set[ListenerName] = {
- val listenersSet = listeners.map(_.listenerName).toSet
- val controllerListenersSet = controllerListeners.map(_.listenerName).toSet
- Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match {
- case None => controllerListenersSet
- case Some(str) =>
- str.split(",").map(_.trim()).filterNot(_.isEmpty).map { str =>
- val listenerName = new ListenerName(str)
- if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName))
- throw new ConfigException(s"${ServerConfigs.EARLY_START_LISTENERS_CONFIG} contains " +
- s"listener ${listenerName.value()}, but this is not contained in " +
- s"${SocketServerConfigs.LISTENERS_CONFIG} or ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}")
- listenerName
- }.toSet
- }
- }
-
- /** ********* Socket Server Configuration ***********/
- val socketSendBufferBytes = getInt(SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG)
- val socketReceiveBufferBytes = getInt(SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG)
- val socketRequestMaxBytes = getInt(SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG)
- val socketListenBacklogSize = getInt(SocketServerConfigs.SOCKET_LISTEN_BACKLOG_SIZE_CONFIG)
- val maxConnectionsPerIp = getInt(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG)
- val maxConnectionsPerIpOverrides: Map[String, Int] =
- getMap(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG, getString(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG)).map { case (k, v) => (k, v.toInt)}
- def maxConnections = getInt(SocketServerConfigs.MAX_CONNECTIONS_CONFIG)
- def maxConnectionCreationRate = getInt(SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG)
- val connectionsMaxIdleMs = getLong(SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG)
- val failedAuthenticationDelayMs = getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG)
-
- /***************** rack configuration **************/
- val rack = Option(getString(ServerConfigs.BROKER_RACK_CONFIG))
- val replicaSelectorClassName = Option(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG))
-
- /** ********* Log Configuration ***********/
- val autoCreateTopicsEnable = getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG)
- val numPartitions = getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG)
- val logDirs: Seq[String] = Csv.parseCsvList(Option(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).getOrElse(getString(ServerLogConfigs.LOG_DIR_CONFIG))).asScala
- def logSegmentBytes = getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG)
- def logFlushIntervalMessages = getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG)
- val logCleanerThreads = getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP)
- def numRecoveryThreadsPerDataDir = getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG)
- val logFlushSchedulerIntervalMs = getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG)
- val logFlushOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
- val logFlushStartOffsetCheckpointIntervalMs = getInt(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG).toLong
- val logCleanupIntervalMs = getLong(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG)
- def logCleanupPolicy = getList(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG)
-
- val offsetsRetentionMinutes = getInt(GroupCoordinatorConfig.OFFSETS_RETENTION_MINUTES_CONFIG)
- val offsetsRetentionCheckIntervalMs = getLong(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG)
- def logRetentionBytes = getLong(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG)
- val logCleanerDedupeBufferSize = getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP)
- val logCleanerDedupeBufferLoadFactor = getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP)
- val logCleanerIoBufferSize = getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP)
- val logCleanerIoMaxBytesPerSecond = getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP)
- def logCleanerDeleteRetentionMs = getLong(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP)
- def logCleanerMinCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP)
- def logCleanerMaxCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP)
- val logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
- def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
- val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
- def logIndexSizeMaxBytes = getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG)
- def logIndexIntervalBytes = getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG)
- def logDeleteDelayMs = getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG)
- def logRollTimeMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG))
- def logRollTimeJitterMillis: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG)).getOrElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG))
- def logFlushIntervalMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG)).getOrElse(getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG))
- def minInSyncReplicas = getInt(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG)
- def logPreAllocateEnable: java.lang.Boolean = getBoolean(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG)
- def logInitialTaskDelayMs: java.lang.Long = Option(getLong(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG)).getOrElse(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT)
-
- // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
- // is passed, `0.10.0-IV0` may be picked)
- @nowarn("cat=deprecation")
- private val logMessageFormatVersionString = getString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG)
-
- /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
- @deprecated("3.0")
- lazy val logMessageFormatVersion =
- if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion))
- MetadataVersion.fromVersionString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT)
- else MetadataVersion.fromVersionString(logMessageFormatVersionString)
-
- def logMessageTimestampType = TimestampType.forName(getString(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG))
-
- /* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details */
- @deprecated("3.6")
- def logMessageTimestampDifferenceMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG)
-
- // In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
- // we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed.
- // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
- @nowarn("cat=deprecation")
- def logMessageTimestampBeforeMaxMs: Long = {
- val messageTimestampBeforeMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG)
- if (messageTimestampBeforeMaxMs != ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT) {
- messageTimestampBeforeMaxMs
- } else {
- logMessageTimestampDifferenceMaxMs
- }
- }
-
- // In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
- // we are using its value if logMessageTimestampAfterMaxMs default value hasn't changed.
- // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
- @nowarn("cat=deprecation")
- def logMessageTimestampAfterMaxMs: Long = {
- val messageTimestampAfterMaxMs: Long = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG)
- if (messageTimestampAfterMaxMs != Long.MaxValue) {
- messageTimestampAfterMaxMs
- } else {
- logMessageTimestampDifferenceMaxMs
- }
- }
-
- def logMessageDownConversionEnable: Boolean = getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG)
-
- def logDirFailureTimeoutMs: Long = getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG)
-
- /** ********* Replication configuration ***********/
- val controllerSocketTimeoutMs: Int = getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG)
- val defaultReplicationFactor: Int = getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG)
- val replicaLagTimeMaxMs = getLong(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG)
- val replicaSocketTimeoutMs = getInt(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG)
- val replicaSocketReceiveBufferBytes = getInt(ReplicationConfigs.REPLICA_SOCKET_RECEIVE_BUFFER_BYTES_CONFIG)
- val replicaFetchMaxBytes = getInt(ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG)
- val replicaFetchWaitMaxMs = getInt(ReplicationConfigs.REPLICA_FETCH_WAIT_MAX_MS_CONFIG)
- val replicaFetchMinBytes = getInt(ReplicationConfigs.REPLICA_FETCH_MIN_BYTES_CONFIG)
- val replicaFetchResponseMaxBytes = getInt(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_CONFIG)
- val replicaFetchBackoffMs = getInt(ReplicationConfigs.REPLICA_FETCH_BACKOFF_MS_CONFIG)
- def numReplicaFetchers = getInt(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG)
- val replicaHighWatermarkCheckpointIntervalMs = getLong(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG)
- val fetchPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
- val producerPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
- val deleteRecordsPurgatoryPurgeIntervalRequests = getInt(ReplicationConfigs.DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG)
- val autoLeaderRebalanceEnable = getBoolean(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG)
- val leaderImbalancePerBrokerPercentage = getInt(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG)
- val leaderImbalanceCheckIntervalSeconds: Long = getLong(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG)
- def uncleanLeaderElectionEnable: java.lang.Boolean = getBoolean(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG)
-
- // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
- // is passed, `0.10.0-IV0` may be picked)
- val interBrokerProtocolVersionString = getString(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)
- val interBrokerProtocolVersion = if (processRoles.isEmpty) {
- MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
- } else {
- if (originals.containsKey(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)) {
- // A user-supplied IBP was given
- val configuredVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString)
- if (!configuredVersion.isKRaftSupported) {
- throw new ConfigException(s"A non-KRaft version $interBrokerProtocolVersionString given for ${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG}. " +
- s"The minimum version is ${MetadataVersion.MINIMUM_KRAFT_VERSION}")
- } else {
- warn(s"${ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG} is deprecated in KRaft mode as of 3.3 and will only " +
- s"be read when first upgrading from a KRaft prior to 3.3. See kafka-storage.sh help for details on setting " +
- s"the metadata.version for a new KRaft cluster.")
- }
- }
- // In KRaft mode, we pin this value to the minimum KRaft-supported version. This prevents inadvertent usage of
- // the static IBP config in broker components running in KRaft mode
- MetadataVersion.MINIMUM_KRAFT_VERSION
- }
-
- /** ********* Controlled shutdown configuration ***********/
- val controlledShutdownMaxRetries = getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG)
- val controlledShutdownRetryBackoffMs = getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG)
- val controlledShutdownEnable = getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG)
-
- /** ********* Feature configuration ***********/
- def isFeatureVersioningSupported = interBrokerProtocolVersion.isFeatureVersioningSupported
-
- /** ********* Group coordinator configuration ***********/
- val groupMinSessionTimeoutMs = getInt(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG)
- val groupMaxSessionTimeoutMs = getInt(GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG)
- val groupInitialRebalanceDelay = getInt(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG)
- val groupMaxSize = getInt(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG)
-
- /** New group coordinator configs */
- val groupCoordinatorRebalanceProtocols = {
- val protocols = getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG)
- .asScala.map(_.toUpperCase).map(GroupType.valueOf).toSet
- if (!protocols.contains(GroupType.CLASSIC)) {
- throw new ConfigException(s"Disabling the '${GroupType.CLASSIC}' protocol is not supported.")
- }
- if (protocols.contains(GroupType.CONSUMER)) {
- if (processRoles.isEmpty) {
- throw new ConfigException(s"The new '${GroupType.CONSUMER}' rebalance protocol is only supported in KRaft cluster.")
- }
- warn(s"The new '${GroupType.CONSUMER}' rebalance protocol is enabled along with the new group coordinator. " +
- "This is part of the preview of KIP-848 and MUST NOT be used in production.")
- }
- protocols
- }
- // The new group coordinator is enabled in two cases: 1) The internal configuration to enable
- // it is explicitly set; or 2) the consumer rebalance protocol is enabled.
- val isNewGroupCoordinatorEnabled = getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG) ||
- groupCoordinatorRebalanceProtocols.contains(GroupType.CONSUMER)
- val groupCoordinatorNumThreads = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG)
- val groupCoordinatorAppendLingerMs = getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG)
-
- /** Consumer group configs */
- val consumerGroupSessionTimeoutMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG)
- val consumerGroupMinSessionTimeoutMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG)
- val consumerGroupMaxSessionTimeoutMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG)
- val consumerGroupHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG)
- val consumerGroupMinHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG)
- val consumerGroupMaxHeartbeatIntervalMs = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG)
- val consumerGroupMaxSize = getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG)
- val consumerGroupAssignors = getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, classOf[ConsumerGroupPartitionAssignor])
- val consumerGroupMigrationPolicy = ConsumerGroupMigrationPolicy.parse(getString(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG))
-
- /** Share group configuration **/
- val isShareGroupEnabled = getBoolean(ShareGroupConfigs.SHARE_GROUP_ENABLE_CONFIG)
- val shareGroupPartitionMaxRecordLocks = getInt(ShareGroupConfigs.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG)
- val shareGroupDeliveryCountLimit = getInt(ShareGroupConfigs.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG)
- val shareGroupMaxGroups = getShort(ShareGroupConfigs.SHARE_GROUP_MAX_GROUPS_CONFIG)
- val shareGroupMaxSize = getShort(ShareGroupConfigs.SHARE_GROUP_MAX_SIZE_CONFIG)
- val shareGroupSessionTimeoutMs = getInt(ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG)
- val shareGroupMinSessionTimeoutMs = getInt(ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG)
- val shareGroupMaxSessionTimeoutMs = getInt(ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG)
- val shareGroupHeartbeatIntervalMs = getInt(ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG)
- val shareGroupMinHeartbeatIntervalMs = getInt(ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG)
- val shareGroupMaxHeartbeatIntervalMs = getInt(ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG)
- val shareGroupRecordLockDurationMs = getInt(ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG)
- val shareGroupMaxRecordLockDurationMs = getInt(ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG)
- val shareGroupMinRecordLockDurationMs = getInt(ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG)
-
- /** ********* Offset management configuration ***********/
- val offsetMetadataMaxSize = getInt(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG)
- val offsetsLoadBufferSize = getInt(GroupCoordinatorConfig.OFFSETS_LOAD_BUFFER_SIZE_CONFIG)
- val offsetsTopicReplicationFactor = getShort(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG)
- val offsetsTopicPartitions = getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG)
- val offsetCommitTimeoutMs = getInt(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG)
- @deprecated("3.8")
- val offsetCommitRequiredAcks = getShort(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG)
- val offsetsTopicSegmentBytes = getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_SEGMENT_BYTES_CONFIG)
- val offsetsTopicCompressionType = Option(getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG)).map(value => CompressionType.forId(value)).orNull
-
- /** ********* Transaction management configuration ***********/
- val transactionalIdExpirationMs = getInt(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG)
- val transactionMaxTimeoutMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_MAX_TIMEOUT_MS_CONFIG)
- val transactionTopicMinISR = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG)
- val transactionsLoadBufferSize = getInt(TransactionLogConfigs.TRANSACTIONS_LOAD_BUFFER_SIZE_CONFIG)
- val transactionTopicReplicationFactor = getShort(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG)
- val transactionTopicPartitions = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG)
- val transactionTopicSegmentBytes = getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG)
- val transactionAbortTimedOutTransactionCleanupIntervalMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG)
- val transactionRemoveExpiredTransactionalIdCleanupIntervalMs = getInt(TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_CONFIG)
-
- def transactionPartitionVerificationEnable = getBoolean(TransactionLogConfigs.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG)
-
- def producerIdExpirationMs = getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_CONFIG)
- val producerIdExpirationCheckIntervalMs = getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG)
-
- /** ********* Metric Configuration **************/
- val metricNumSamples = getInt(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG)
- val metricSampleWindowMs = getLong(MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG)
- val metricRecordingLevel = getString(MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG)
-
- /** ********* Kafka Client Telemetry Metrics Configuration ***********/
- val clientTelemetryMaxBytes: Int = getInt(MetricConfigs.CLIENT_TELEMETRY_MAX_BYTES_CONFIG)
-
- /** ********* SSL/SASL Configuration **************/
- // Security configs may be overridden for listeners, so it is not safe to use the base values
- // Hence the base SSL/SASL configs are not fields of KafkaConfig, listener configs should be
- // retrieved using KafkaConfig#valuesWithPrefixOverride
- private def saslEnabledMechanisms(listenerName: ListenerName): Set[String] = {
- val value = valuesWithPrefixOverride(listenerName.configPrefix).get(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG)
- if (value != null)
- value.asInstanceOf[util.List[String]].asScala.toSet
- else
- Set.empty[String]
- }
-
- def interBrokerListenerName = getInterBrokerListenerNameAndSecurityProtocol._1
- def interBrokerSecurityProtocol = getInterBrokerListenerNameAndSecurityProtocol._2
- def controlPlaneListenerName = getControlPlaneListenerNameAndSecurityProtocol.map { case (listenerName, _) => listenerName }
- def controlPlaneSecurityProtocol = getControlPlaneListenerNameAndSecurityProtocol.map { case (_, securityProtocol) => securityProtocol }
- def saslMechanismInterBrokerProtocol = getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG)
- val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion.isSaslInterBrokerHandshakeRequestEnabled
-
- /** ********* DelegationToken Configuration **************/
- val delegationTokenSecretKey = Option(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG))
- .getOrElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG))
- val tokenAuthEnabled = delegationTokenSecretKey != null && delegationTokenSecretKey.value.nonEmpty
- val delegationTokenMaxLifeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG)
- val delegationTokenExpiryTimeMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG)
- val delegationTokenExpiryCheckIntervalMs = getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG)
-
- /** ********* Password encryption configuration for dynamic configs *********/
- def passwordEncoderSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG))
- def passwordEncoderOldSecret = Option(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_OLD_SECRET_CONFIG))
- def passwordEncoderCipherAlgorithm = getString(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG)
- def passwordEncoderKeyFactoryAlgorithm = getString(PasswordEncoderConfigs.PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG)
- def passwordEncoderKeyLength = getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG)
- def passwordEncoderIterations = getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG)
-
- /** ********* Quota Configuration **************/
- val numQuotaSamples = getInt(QuotaConfigs.NUM_QUOTA_SAMPLES_CONFIG)
- val quotaWindowSizeSeconds = getInt(QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
- val numReplicationQuotaSamples = getInt(QuotaConfigs.NUM_REPLICATION_QUOTA_SAMPLES_CONFIG)
- val replicationQuotaWindowSizeSeconds = getInt(QuotaConfigs.REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
- val numAlterLogDirsReplicationQuotaSamples = getInt(QuotaConfigs.NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES_CONFIG)
- val alterLogDirsReplicationQuotaWindowSizeSeconds = getInt(QuotaConfigs.ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
- val numControllerQuotaSamples = getInt(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG)
- val controllerQuotaWindowSizeSeconds = getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG)
-
- /** ********* Fetch Configuration **************/
- val maxIncrementalFetchSessionCacheSlots = getInt(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG)
- val fetchMaxBytes = getInt(ServerConfigs.FETCH_MAX_BYTES_CONFIG)
-
- /** ********* Request Limit Configuration ***********/
- val maxRequestPartitionSizeLimit = getInt(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG)
-
- val deleteTopicEnable = getBoolean(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG)
- def compressionType = getString(ServerConfigs.COMPRESSION_TYPE_CONFIG)
-
- def gzipCompressionLevel = getInt(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG)
- def lz4CompressionLevel = getInt(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG)
- def zstdCompressionLevel = getInt(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG)
-
- /** ********* Raft Quorum Configuration *********/
- val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG)
- val quorumBootstrapServers = getList(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG)
- val quorumElectionTimeoutMs = getInt(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG)
- val quorumFetchTimeoutMs = getInt(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG)
- val quorumElectionBackoffMs = getInt(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG)
- val quorumLingerMs = getInt(QuorumConfig.QUORUM_LINGER_MS_CONFIG)
- val quorumRequestTimeoutMs = getInt(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG)
- val quorumRetryBackoffMs = getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG)
-
- /** Internal Configurations **/
- val unstableApiVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG)
- val unstableFeatureVersionsEnabled = getBoolean(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG)
def addReconfigurable(reconfigurable: Reconfigurable): Unit = {
dynamicConfig.addReconfigurable(reconfigurable)
@@ -734,156 +266,16 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
dynamicConfig.removeReconfigurable(reconfigurable)
}
- def logRetentionTimeMillis: Long = {
- val millisInMinute = 60L * 1000L
- val millisInHour = 60L * millisInMinute
-
- val millis: java.lang.Long =
- Option(getLong(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)).getOrElse(
- Option(getInt(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG)) match {
- case Some(mins) => millisInMinute * mins
- case None => getInt(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG) * millisInHour
- })
-
- if (millis < 0) return -1
- millis
- }
-
- private def getMap(propName: String, propValue: String): Map[String, String] = {
- try {
- Csv.parseCsvMap(propValue).asScala
- } catch {
- case e: Exception => throw new IllegalArgumentException("Error parsing configuration property '%s': %s".format(propName, e.getMessage))
- }
- }
-
- def listeners: Seq[EndPoint] =
- CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
-
- def controllerListenerNames: Seq[String] = {
- val value = Option(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)).getOrElse("")
- if (value.isEmpty) {
- Seq.empty
- } else {
- value.split(",")
- }
- }
-
- def controllerListeners: Seq[EndPoint] =
- listeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
-
- def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG)
-
- def controlPlaneListener: Option[EndPoint] = {
- controlPlaneListenerName.map { listenerName =>
- listeners.filter(endpoint => endpoint.listenerName.value() == listenerName.value()).head
- }
- }
-
- def dataPlaneListeners: Seq[EndPoint] = {
- listeners.filterNot { listener =>
- val name = listener.listenerName.value()
- name.equals(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)) ||
- controllerListenerNames.contains(name)
- }
- }
-
- // Use advertised listeners if defined, fallback to listeners otherwise
- def effectiveAdvertisedListeners: Seq[EndPoint] = {
- val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
- if (advertisedListenersProp != null)
- CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
- else
- listeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value()))
- }
-
- private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = {
- Option(getString(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG)) match {
- case Some(_) if originals.containsKey(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG) =>
- throw new ConfigException(s"Only one of ${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} and " +
- s"${ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG} should be set.")
- case Some(name) =>
- val listenerName = ListenerName.normalised(name)
- val securityProtocol = effectiveListenerSecurityProtocolMap.getOrElse(listenerName,
- throw new ConfigException(s"Listener with name ${listenerName.value} defined in " +
- s"${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG}."))
- (listenerName, securityProtocol)
- case None =>
- val securityProtocol = getSecurityProtocol(getString(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG),
- ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG)
- (ListenerName.forSecurityProtocol(securityProtocol), securityProtocol)
- }
- }
-
- private def getControlPlaneListenerNameAndSecurityProtocol: Option[(ListenerName, SecurityProtocol)] = {
- Option(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)) match {
- case Some(name) =>
- val listenerName = ListenerName.normalised(name)
- val securityProtocol = effectiveListenerSecurityProtocolMap.getOrElse(listenerName,
- throw new ConfigException(s"Listener with ${listenerName.value} defined in " +
- s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} not found in ${SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG}."))
- Some(listenerName, securityProtocol)
-
- case None => None
- }
- }
-
- private def getSecurityProtocol(protocolName: String, configName: String): SecurityProtocol = {
- try SecurityProtocol.forName(protocolName)
- catch {
- case _: IllegalArgumentException =>
- throw new ConfigException(s"Invalid security protocol `$protocolName` defined in $configName")
- }
- }
-
- def effectiveListenerSecurityProtocolMap: Map[ListenerName, SecurityProtocol] = {
- val mapValue = getMap(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG))
- .map { case (listenerName, protocolName) =>
- ListenerName.normalised(listenerName) -> getSecurityProtocol(protocolName, SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)
- }
- if (usesSelfManagedQuorum && !originals.containsKey(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)) {
- // Nothing was specified explicitly for listener.security.protocol.map, so we are using the default value,
- // and we are using KRaft.
- // Add PLAINTEXT mappings for controller listeners as long as there is no SSL or SASL_{PLAINTEXT,SSL} in use
- def isSslOrSasl(name: String): Boolean = name.equals(SecurityProtocol.SSL.name) || name.equals(SecurityProtocol.SASL_SSL.name) || name.equals(SecurityProtocol.SASL_PLAINTEXT.name)
- // check controller listener names (they won't appear in listeners when process.roles=broker)
- // as well as listeners for occurrences of SSL or SASL_*
- if (controllerListenerNames.exists(isSslOrSasl) ||
- Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(EndPoint.parseListenerName(listenerValue)))) {
- mapValue // don't add default mappings since we found something that is SSL or SASL_*
- } else {
- // add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT
- mapValue ++ controllerListenerNames.filterNot(SecurityProtocol.PLAINTEXT.name.equals(_)).map(
- new ListenerName(_) -> SecurityProtocol.PLAINTEXT)
- }
- } else {
- mapValue
- }
- }
-
- // Topic IDs are used with all self-managed quorum clusters and ZK cluster with IBP greater than or equal to 2.8
- def usesTopicId: Boolean =
- usesSelfManagedQuorum || interBrokerProtocolVersion.isTopicIdsSupported
-
- def logLocalRetentionBytes: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP)
-
- def logLocalRetentionMs: java.lang.Long = getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP)
-
- def remoteFetchMaxWaitMs = getInt(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP)
-
- def remoteLogIndexFileCacheTotalSizeBytes: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP)
-
- def remoteLogManagerCopyMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP)
-
- def remoteLogManagerFetchMaxBytesPerSecond: Long = getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP)
-
validateValues()
- @nowarn("cat=deprecation")
+ /**
+ * This method is kept here for now to reduce the conflict between the work of moving KafkaConfig out and removing ZK.
+ * Any validations related to ZK mode will stay here until we cleanup the ZK code.
+ * Other validations will be defined in {@link org.apache.kafka.server.config.KafkaConfigValidator}
+ */
private def validateValues(): Unit = {
- if (nodeId != brokerId) {
- throw new ConfigException(s"You must set `${KRaftConfigs.NODE_ID_CONFIG}` to the same value as `${ServerConfigs.BROKER_ID_CONFIG}`.")
- }
+ configValidator.validateNodeAndBrokerId()
+
if (requiresZookeeper) {
if (zkConnect == null) {
throw new ConfigException(s"Missing required configuration `${ZkConfigs.ZK_CONNECT_CONFIG}` which has no default value.")
@@ -905,116 +297,24 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
}
}
}
- require(logRollTimeMillis >= 1, "log.roll.ms must be greater than or equal to 1")
- require(logRollTimeJitterMillis >= 0, "log.roll.jitter.ms must be greater than or equal to 0")
- require(logRetentionTimeMillis >= 1 || logRetentionTimeMillis == -1, "log.retention.ms must be unlimited (-1) or, greater than or equal to 1")
- require(logDirs.nonEmpty, "At least one log directory must be defined via log.dirs or log.dir.")
- require(logCleanerDedupeBufferSize / logCleanerThreads > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.")
- require(replicaFetchWaitMaxMs <= replicaSocketTimeoutMs, "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" +
- " to prevent unnecessary socket timeouts")
- require(replicaFetchWaitMaxMs <= replicaLagTimeMaxMs, "replica.fetch.wait.max.ms should always be less than or equal to replica.lag.time.max.ms" +
- " to prevent frequent changes in ISR")
- require(offsetCommitRequiredAcks >= -1 && offsetCommitRequiredAcks <= offsetsTopicReplicationFactor,
- "offsets.commit.required.acks must be greater or equal -1 and less or equal to offsets.topic.replication.factor")
- val advertisedListenerNames = effectiveAdvertisedListeners.map(_.listenerName).toSet
+
+ configValidator.validateLogConfig()
+ configValidator.validateReplicaFetchConfigs()
+ configValidator.validateOffsetCommitAcks()
// validate KRaft-related configs
- val voterIds = QuorumConfig.parseVoterIds(quorumVoters)
- def validateNonEmptyQuorumVotersForKRaft(): Unit = {
- if (voterIds.isEmpty) {
- 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 (voterIds.isEmpty) {
- throw new ConfigException(s"If using ${KRaftConfigs.MIGRATION_ENABLED_CONFIG}, ${QuorumConfig.QUORUM_VOTERS_CONFIG} must contain a parseable set of voters.")
- }
- }
- def validateControlPlaneListenerEmptyForKRaft(): Unit = {
- require(controlPlaneListenerName.isEmpty,
- s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} is not supported in KRaft mode.")
- }
- def validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker(): Unit = {
- require(!advertisedListenerNames.exists(aln => controllerListenerNames.contains(aln.value())),
- 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(voterIds.contains(nodeId),
- 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}=${voterIds.asScala.toSet}")
- }
- def validateControllerListenerExistsForKRaftController(): Unit = {
- require(controllerListeners.nonEmpty,
- 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"${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 ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG}?" else ""))
- }
- if (processRoles == Set(ProcessRole.BrokerRole)) {
- // KRaft broker-only
- validateNonEmptyQuorumVotersForKRaft()
- validateControlPlaneListenerEmptyForKRaft()
- validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker()
- // nodeId must not appear in controller.quorum.voters
- require(!voterIds.contains(nodeId),
- 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}=${voterIds.asScala.toSet}")
- // controller.listener.names must be non-empty...
- require(controllerListenerNames.nonEmpty,
- 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"${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"${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"${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)) {
- // KRaft controller-only
- validateNonEmptyQuorumVotersForKRaft()
- validateControlPlaneListenerEmptyForKRaft()
- // 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 ${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 ${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} when ${KRaftConfigs.PROCESS_ROLES_CONFIG}=controller"
- )
- validateControllerQuorumVotersMustContainNodeIdForKRaftController()
- validateControllerListenerExistsForKRaftController()
- validateControllerListenerNamesMustAppearInListenersForKRaftController()
+ if (processRoles == Collections.singleton(ProcessRole.BrokerRole)) {
+ configValidator.validateKraftBrokerConfig()
+ } else if (processRoles == Collections.singleton(ProcessRole.ControllerRole)) {
+ configValidator.validateKraftControllerConfig()
} else if (isKRaftCombinedMode) {
- // KRaft combined broker and controller
- validateNonEmptyQuorumVotersForKRaft()
- validateControlPlaneListenerEmptyForKRaft()
- validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker()
- validateControllerQuorumVotersMustContainNodeIdForKRaftController()
- validateControllerListenerExistsForKRaftController()
- validateControllerListenerNamesMustAppearInListenersForKRaftController()
- validateAdvertisedListenersNonEmptyForBroker()
+ configValidator.validateKraftCombinedModeConfig()
} else {
// ZK-based
if (migrationEnabled) {
- validateNonEmptyQuorumVotersForMigration()
- require(controllerListenerNames.nonEmpty,
- s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames.asJava}")
+ configValidator.validateNonEmptyQuorumVotersForMigration()
+ require(controllerListenerNames.asScala.nonEmpty,
+ s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must not be empty when running in ZooKeeper migration mode: ${controllerListenerNames}")
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) {
@@ -1025,194 +325,18 @@ 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"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} 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}")
}
- validateAdvertisedListenersNonEmptyForBroker()
+ configValidator.validateAdvertisedListenersNonEmptyForBroker()
}
- val listenerNames = listeners.map(_.listenerName).toSet
- if (processRoles.isEmpty || processRoles.contains(ProcessRole.BrokerRole)) {
- // validations for all broker setups (i.e. ZooKeeper and KRaft broker-only and KRaft co-located)
- validateAdvertisedListenersNonEmptyForBroker()
- require(advertisedListenerNames.contains(interBrokerListenerName),
- s"${ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG} must be a listener name defined in ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG}. " +
- s"The valid options based on currently configured listeners are ${advertisedListenerNames.map(_.value).mkString(",")}")
- require(advertisedListenerNames.subsetOf(listenerNames),
- s"${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} listener names must be equal to or a subset of the ones defined in ${SocketServerConfigs.LISTENERS_CONFIG}. " +
- s"Found ${advertisedListenerNames.map(_.value).mkString(",")}. The valid options based on the current configuration " +
- s"are ${listenerNames.map(_.value).mkString(",")}"
- )
- }
-
- require(!effectiveAdvertisedListeners.exists(endpoint => endpoint.host=="0.0.0.0"),
- s"${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG} cannot use the nonroutable meta-address 0.0.0.0. "+
- s"Use a routable IP address.")
-
- // validate control.plane.listener.name config
- if (controlPlaneListenerName.isDefined) {
- require(advertisedListenerNames.contains(controlPlaneListenerName.get),
- s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG} must be a listener name defined in ${SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG}. " +
- s"The valid options based on currently configured listeners are ${advertisedListenerNames.map(_.value).mkString(",")}")
- // controlPlaneListenerName should be different from interBrokerListenerName
- require(!controlPlaneListenerName.get.value().equals(interBrokerListenerName.value()),
- s"${SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG}, when defined, should have a different value from the inter broker listener name. " +
- s"Currently they both have the value ${controlPlaneListenerName.get}")
- }
-
- val messageFormatVersion = new MessageFormatVersion(logMessageFormatVersionString, interBrokerProtocolVersionString)
- if (messageFormatVersion.shouldWarn)
- warn(createBrokerWarningMessage)
-
- val recordVersion = logMessageFormatVersion.highestSupportedRecordVersion
- require(interBrokerProtocolVersion.highestSupportedRecordVersion().value >= recordVersion.value,
- s"log.message.format.version $logMessageFormatVersionString can only be used when inter.broker.protocol.version " +
- s"is set to version ${MetadataVersion.minSupportedFor(recordVersion).shortVersion} or higher")
-
- if (offsetsTopicCompressionType == CompressionType.ZSTD)
- require(interBrokerProtocolVersion.highestSupportedRecordVersion().value >= IBP_2_1_IV0.highestSupportedRecordVersion().value,
- "offsets.topic.compression.codec zstd can only be used when inter.broker.protocol.version " +
- s"is set to version ${IBP_2_1_IV0.shortVersion} or higher")
-
- val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL
- require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM,
- s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString")
- require(!interBrokerUsesSasl || saslEnabledMechanisms(interBrokerListenerName).contains(saslMechanismInterBrokerProtocol),
- s"${BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG} must be included in ${BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG} when SASL is used for inter-broker communication")
- require(queuedMaxBytes <= 0 || queuedMaxBytes >= socketRequestMaxBytes,
- s"${ServerConfigs.QUEUED_MAX_BYTES_CONFIG} must be larger or equal to ${SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG}")
-
- if (maxConnectionsPerIp == 0)
- require(maxConnectionsPerIpOverrides.nonEmpty, s"${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG} can be set to zero only if" +
- s" ${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG} property is set.")
-
- val invalidAddresses = maxConnectionsPerIpOverrides.keys.filterNot(address => Utils.validHostPattern(address))
- if (invalidAddresses.nonEmpty)
- throw new IllegalArgumentException(s"${SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG} contains invalid addresses : ${invalidAddresses.mkString(",")}")
-
- if (connectionsMaxIdleMs >= 0)
- require(failedAuthenticationDelayMs < connectionsMaxIdleMs,
- s"${SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG}=$failedAuthenticationDelayMs should always be less than" +
- s" ${SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG}=$connectionsMaxIdleMs to prevent failed" +
- s" authentication responses from timing out")
-
- val principalBuilderClass = getClass(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG)
- require(principalBuilderClass != null, s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must be non-null")
- require(classOf[KafkaPrincipalSerde].isAssignableFrom(principalBuilderClass),
- s"${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG} must implement KafkaPrincipalSerde")
-
- // New group coordinator configs validation.
- require(consumerGroupMaxHeartbeatIntervalMs >= consumerGroupMinHeartbeatIntervalMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG} must be greater than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG}")
- require(consumerGroupHeartbeatIntervalMs >= consumerGroupMinHeartbeatIntervalMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG} must be greater than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG}")
- require(consumerGroupHeartbeatIntervalMs <= consumerGroupMaxHeartbeatIntervalMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG} must be less than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG}")
-
- require(consumerGroupMaxSessionTimeoutMs >= consumerGroupMinSessionTimeoutMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG} must be greater than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG}")
- require(consumerGroupSessionTimeoutMs >= consumerGroupMinSessionTimeoutMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG} must be greater than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG}")
- require(consumerGroupSessionTimeoutMs <= consumerGroupMaxSessionTimeoutMs,
- s"${GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG} must be less than or equals " +
- s"to ${GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG}")
-
- require(shareGroupMaxHeartbeatIntervalMs >= shareGroupMinHeartbeatIntervalMs,
- s"${ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG}")
- require(shareGroupHeartbeatIntervalMs >= shareGroupMinHeartbeatIntervalMs,
- s"${ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG}")
- require(shareGroupHeartbeatIntervalMs <= shareGroupMaxHeartbeatIntervalMs,
- s"${ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG} must be less than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG}")
-
- require(shareGroupMaxSessionTimeoutMs >= shareGroupMinSessionTimeoutMs,
- s"${ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG}")
- require(shareGroupSessionTimeoutMs >= shareGroupMinSessionTimeoutMs,
- s"${ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG}")
- require(shareGroupSessionTimeoutMs <= shareGroupMaxSessionTimeoutMs,
- s"${ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG} must be less than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG}")
-
- require(shareGroupMaxRecordLockDurationMs >= shareGroupMinRecordLockDurationMs,
- s"${ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG}")
- require(shareGroupRecordLockDurationMs >= shareGroupMinRecordLockDurationMs,
- s"${ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG}")
- require(shareGroupMaxRecordLockDurationMs >= shareGroupRecordLockDurationMs,
- s"${ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG} must be greater than or equals " +
- s"to ${ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG}")
-
- if (originals.containsKey(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG)) {
- warn(s"${GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG} is deprecated and it will be removed in Apache Kafka 4.0.")
- }
- }
-
- /**
- * Validate some configurations for new MetadataVersion. A new MetadataVersion can take place when
- * a FeatureLevelRecord for "metadata.version" is read from the cluster metadata.
- */
- def validateWithMetadataVersion(metadataVersion: MetadataVersion): Unit = {
- if (processRoles.contains(ProcessRole.BrokerRole) && logDirs.size > 1) {
- require(metadataVersion.isDirectoryAssignmentSupported,
- s"Multiple log directories (aka JBOD) are not supported in the current MetadataVersion ${metadataVersion}. " +
- s"Need ${MetadataVersion.IBP_3_7_IV2} or higher")
- }
- }
-
- /**
- * Copy the subset of properties that are relevant to Logs. The individual properties
- * are listed here since the names are slightly different in each Config class...
- */
- @nowarn("cat=deprecation")
- def extractLogConfigMap: java.util.Map[String, Object] = {
- val logProps = new java.util.HashMap[String, Object]()
- logProps.put(TopicConfig.SEGMENT_BYTES_CONFIG, logSegmentBytes)
- logProps.put(TopicConfig.SEGMENT_MS_CONFIG, logRollTimeMillis)
- logProps.put(TopicConfig.SEGMENT_JITTER_MS_CONFIG, logRollTimeJitterMillis)
- logProps.put(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, logIndexSizeMaxBytes)
- logProps.put(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, logFlushIntervalMessages)
- logProps.put(TopicConfig.FLUSH_MS_CONFIG, logFlushIntervalMs)
- logProps.put(TopicConfig.RETENTION_BYTES_CONFIG, logRetentionBytes)
- logProps.put(TopicConfig.RETENTION_MS_CONFIG, logRetentionTimeMillis: java.lang.Long)
- logProps.put(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, messageMaxBytes)
- logProps.put(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, logIndexIntervalBytes)
- logProps.put(TopicConfig.DELETE_RETENTION_MS_CONFIG, logCleanerDeleteRetentionMs)
- logProps.put(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, logCleanerMinCompactionLagMs)
- logProps.put(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, logCleanerMaxCompactionLagMs)
- logProps.put(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, logDeleteDelayMs)
- logProps.put(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, logCleanerMinCleanRatio)
- logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy)
- logProps.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas)
- logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType)
- logProps.put(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG, gzipCompressionLevel)
- logProps.put(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG, lz4CompressionLevel)
- logProps.put(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG, zstdCompressionLevel)
- logProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable)
- logProps.put(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable)
- logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)
- logProps.put(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType.name)
- logProps.put(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, logMessageTimestampDifferenceMaxMs: java.lang.Long)
- logProps.put(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, logMessageTimestampBeforeMaxMs: java.lang.Long)
- logProps.put(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs: java.lang.Long)
- logProps.put(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable: java.lang.Boolean)
- logProps.put(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, logLocalRetentionMs)
- logProps.put(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, logLocalRetentionBytes)
- logProps
- }
-
- @nowarn("cat=deprecation")
- private def createBrokerWarningMessage: String = {
- s"Broker configuration ${ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG} with value $logMessageFormatVersionString is ignored " +
- s"because the inter-broker protocol version `$interBrokerProtocolVersionString` is greater or equal than 3.0. " +
- "This configuration is deprecated and it will be removed in Apache Kafka 4.0."
+ configValidator.validateListenerNames()
+ configValidator.validateMessageFormatConfigs()
+ configValidator.validateCompressionConfig()
+ configValidator.validateInterBrokerSecurityConfig()
+ configValidator.validateQueueMaxByte()
+ configValidator.validateConnectionConfigs()
+ configValidator.validateNewGroupCoordinatorConfigs()
+ configValidator.validateSharedGroupConfigs
}
}
diff --git a/core/src/main/scala/kafka/server/KafkaRaftServer.scala b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
index ecb757c1a89ba..bbfa777fbea55 100644
--- a/core/src/main/scala/kafka/server/KafkaRaftServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaRaftServer.scala
@@ -136,7 +136,7 @@ object KafkaRaftServer {
// Load and verify the original ensemble.
val loader = new MetaPropertiesEnsemble.Loader()
loader.addMetadataLogDir(config.metadataLogDir)
- config.logDirs.foreach(loader.addLogDir)
+ config.logDirs.forEach(loader.addLogDir)
val initialMetaPropsEnsemble = loader.load()
val verificationFlags = util.EnumSet.of(REQUIRE_AT_LEAST_ONE_VALID, REQUIRE_METADATA_LOG_DIR)
initialMetaPropsEnsemble.verify(Optional.empty(), OptionalInt.of(config.nodeId), verificationFlags)
diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala
index 3154ab1c344eb..86a863acbcac2 100755
--- a/core/src/main/scala/kafka/server/KafkaServer.scala
+++ b/core/src/main/scala/kafka/server/KafkaServer.scala
@@ -6,7 +6,7 @@
* (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
+ * 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,
@@ -30,7 +30,7 @@ import kafka.raft.KafkaRaftManager
import kafka.server.metadata.{OffsetTrackingListener, ZkConfigRepository, ZkMetadataCache}
import kafka.utils._
import kafka.zk.{AdminZkClient, BrokerInfo, KafkaZkClient}
-import org.apache.kafka.clients.{ApiVersions, ManualMetadataUpdater, MetadataRecoveryStrategy, NetworkClient, NetworkClientUtils}
+import org.apache.kafka.clients._
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.message.ApiMessageType.ListenerType
@@ -74,7 +74,7 @@ import java.util.concurrent._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
import java.util.{Optional, OptionalInt, OptionalLong}
import scala.collection.{Map, Seq}
-import scala.compat.java8.OptionConverters.RichOptionForJava8
+import scala.compat.java8.OptionConverters.{RichOptionForJava8, RichOptionalGeneric}
import scala.jdk.CollectionConverters._
object KafkaServer {
@@ -109,11 +109,11 @@ object KafkaServer {
* to start up and shutdown a single Kafka node.
*/
class KafkaServer(
- val config: KafkaConfig,
- time: Time = Time.SYSTEM,
- threadNamePrefix: Option[String] = None,
- enableForwarding: Boolean = false
-) extends KafkaBroker with Server {
+ val config: KafkaConfig,
+ time: Time = Time.SYSTEM,
+ threadNamePrefix: Option[String] = None,
+ enableForwarding: Boolean = false
+ ) extends KafkaBroker with Server {
private val startupComplete = new AtomicBoolean(false)
private val isShuttingDown = new AtomicBoolean(false)
@@ -237,7 +237,7 @@ class KafkaServer(
/* load metadata */
val initialMetaPropsEnsemble = {
val loader = new MetaPropertiesEnsemble.Loader()
- config.logDirs.foreach(loader.addLogDir)
+ config.logDirs.forEach(loader.addLogDir)
if (config.migrationEnabled) {
loader.addMetadataLogDir(config.metadataLogDir)
}
@@ -257,7 +257,7 @@ class KafkaServer(
initialMetaPropsEnsemble.verify(Optional.of(_clusterId), verificationId, verificationFlags)
/* generate brokerId */
- config.brokerId = getOrGenerateBrokerId(initialMetaPropsEnsemble)
+ config.brokerId(getOrGenerateBrokerId(initialMetaPropsEnsemble))
logContext = new LogContext(s"[KafkaServer id=${config.brokerId}] ")
this.logIdent = logContext.logPrefix
@@ -405,7 +405,7 @@ class KafkaServer(
val brokerEpoch = zkClient.registerBroker(brokerInfo)
/* start token manager */
- tokenManager = new DelegationTokenManagerZk(config, tokenCache, time , zkClient)
+ tokenManager = new DelegationTokenManagerZk(config, tokenCache, time, zkClient)
tokenManager.startup()
/* start kafka controller */
@@ -458,19 +458,21 @@ class KafkaServer(
raftManager.startup()
val networkListeners = new ListenerCollection()
- config.effectiveAdvertisedListeners.foreach { ep =>
+ config.effectiveAdvertisedListeners.forEach { e => {
+ val ep = EndPoint.fromJava(e)
networkListeners.add(new Listener().
setHost(if (Utils.isBlank(ep.host)) InetAddress.getLocalHost.getCanonicalHostName else ep.host).
setName(ep.listenerName.value()).
setPort(if (ep.port == 0) socketServer.boundPort(ep.listenerName) else ep.port).
setSecurityProtocol(ep.securityProtocol.id))
}
+ }
// Even though ZK brokers don't use "metadata.version" feature, we send our IBP here as part of the broker registration
// so the KRaft controller can verify that all brokers are on the same IBP before starting the migration.
val ibpAsFeature =
- java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME,
- VersionRange.of(config.interBrokerProtocolVersion.featureLevel(), config.interBrokerProtocolVersion.featureLevel()))
+ java.util.Collections.singletonMap(MetadataVersion.FEATURE_NAME,
+ VersionRange.of(config.interBrokerProtocolVersion.featureLevel(), config.interBrokerProtocolVersion.featureLevel()))
lifecycleManager.start(
() => listener.highestOffset,
@@ -528,7 +530,7 @@ class KafkaServer(
)
/* Get the authorizer and initialize it if one is specified.*/
- authorizer = config.createNewAuthorizer()
+ authorizer = config.createNewAuthorizer().asScala
authorizer.foreach(_.configure(config.originals))
val authorizerFutures: Map[Endpoint, CompletableFuture[Void]] = authorizer match {
case Some(authZ) =>
@@ -611,10 +613,10 @@ class KafkaServer(
/* start dynamic config manager */
dynamicConfigHandlers = Map[String, ConfigHandler](ConfigType.TOPIC -> new TopicConfigHandler(replicaManager, config, quotaManagers, Some(kafkaController)),
- ConfigType.CLIENT -> new ClientIdConfigHandler(quotaManagers),
- ConfigType.USER -> new UserConfigHandler(quotaManagers, credentialProvider),
- ConfigType.BROKER -> new BrokerConfigHandler(config, quotaManagers),
- ConfigType.IP -> new IpConfigHandler(socketServer.connectionQuotas))
+ ConfigType.CLIENT -> new ClientIdConfigHandler(quotaManagers),
+ ConfigType.USER -> new UserConfigHandler(quotaManagers, credentialProvider),
+ ConfigType.BROKER -> new BrokerConfigHandler(config, quotaManagers),
+ ConfigType.IP -> new IpConfigHandler(socketServer.connectionQuotas))
// Create the config manager. start listening to notifications
dynamicConfigManager = new ZkConfigManager(zkClient, dynamicConfigHandlers)
@@ -690,14 +692,14 @@ class KafkaServer(
}
protected def createRemoteLogManager(): Option[RemoteLogManager] = {
- if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled()) {
- Some(new RemoteLogManager(config, config.brokerId, config.logDirs.head, clusterId, time,
+ if (config.remoteLogManagerConfig.isRemoteStorageSystemEnabled) {
+ Some(new RemoteLogManager(config, config.brokerId, config.logDirs.asScala.head, clusterId, time,
(tp: TopicPartition) => logManager.getLog(tp).asJava,
(tp: TopicPartition, remoteLogStartOffset: java.lang.Long) => {
logManager.getLog(tp).foreach { log =>
log.updateLogStartOffsetFromRemoteTier(remoteLogStartOffset)
}
- },
+ },
brokerTopicStats, metrics))
} else {
None
@@ -748,14 +750,15 @@ class KafkaServer(
}
def createBrokerInfo: BrokerInfo = {
- val endPoints = config.effectiveAdvertisedListeners.map(e => s"${e.host}:${e.port}")
+ val effectiveAdvertisedListeners = config.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava)
+ val endPoints = effectiveAdvertisedListeners.map(e => s"${e.host}:${e.port}")
zkClient.getAllBrokersInCluster.filter(_.id != config.brokerId).foreach { broker =>
val commonEndPoints = broker.endPoints.map(e => s"${e.host}:${e.port}").intersect(endPoints)
require(commonEndPoints.isEmpty, s"Configured end points ${commonEndPoints.mkString(",")} in" +
s" advertised listeners are already registered by broker ${broker.id}")
}
- val listeners = config.effectiveAdvertisedListeners.map { endpoint =>
+ val listeners = effectiveAdvertisedListeners.map { endpoint =>
if (endpoint.port == 0)
endpoint.copy(port = socketServer.boundPort(endpoint.listenerName))
else
@@ -772,7 +775,7 @@ class KafkaServer(
val jmxPort = System.getProperty("com.sun.management.jmxremote.port", "-1").toInt
BrokerInfo(
- Broker(config.brokerId, updatedEndpoints, config.rack, brokerFeatures.supportedFeatures),
+ Broker(config.brokerId, updatedEndpoints, config.rack.asScala, brokerFeatures.supportedFeatures),
config.interBrokerProtocolVersion,
jmxPort)
}
@@ -845,7 +848,7 @@ class KafkaServer(
// 1. Find the controller and establish a connection to it.
// If the controller id or the broker registration are missing, we sleep and retry (if there are remaining retries)
metadataCache.getControllerId match {
- case Some(controllerId: ZkCachedControllerId) =>
+ case Some(controllerId: ZkCachedControllerId) =>
metadataCache.getAliveBrokerNode(controllerId.id, config.interBrokerListenerName) match {
case Some(broker) =>
// if this is the first attempt, if the controller has changed or if an exception was thrown in a previous
@@ -882,10 +885,10 @@ class KafkaServer(
else 3
val controlledShutdownRequest = new ControlledShutdownRequest.Builder(
- new ControlledShutdownRequestData()
- .setBrokerId(config.brokerId)
- .setBrokerEpoch(kafkaController.brokerEpoch),
- controlledShutdownApiVersion)
+ new ControlledShutdownRequestData()
+ .setBrokerId(config.brokerId)
+ .setBrokerEpoch(kafkaController.brokerEpoch),
+ controlledShutdownApiVersion)
val request = networkClient.newClientRequest(prevController.idString, controlledShutdownRequest,
time.milliseconds(), true)
val clientResponse = NetworkClientUtils.sendAndReceive(networkClient, request, time)
@@ -913,7 +916,7 @@ class KafkaServer(
ioException = true
warn("Error during controlled shutdown, possibly because leader movement took longer than the " +
s"configured controller.socket.timeout.ms and/or request.timeout.ms: ${ioe.getMessage}")
- // ignore and try again
+ // ignore and try again
}
}
if (!shutdownSucceeded && remainingRetries > 0) {
@@ -1096,7 +1099,8 @@ class KafkaServer(
/** Return advertised listeners with the bound port (this may differ from the configured port if the latter is `0`). */
def advertisedListeners: Seq[EndPoint] = {
- config.effectiveAdvertisedListeners.map { endPoint =>
+ config.effectiveAdvertisedListeners.asScala.map { e =>
+ val endPoint = EndPoint.fromJava(e)
endPoint.copy(port = boundPort(endPoint.listenerName))
}
}
@@ -1123,10 +1127,10 @@ class KafkaServer(
}
/**
- * Return a sequence id generated by updating the broker sequence id path in ZK.
- * Users can provide brokerId in the config. To avoid conflicts between ZK generated
- * sequence id and configured brokerId, we increment the generated sequence id by KafkaConfig.MaxReservedBrokerId.
- */
+ * Return a sequence id generated by updating the broker sequence id path in ZK.
+ * Users can provide brokerId in the config. To avoid conflicts between ZK generated
+ * sequence id and configured brokerId, we increment the generated sequence id by KafkaConfig.MaxReservedBrokerId.
+ */
private def generateBrokerId(): Int = {
try {
zkClient.generateBrokerSequenceId() + config.maxReservedBrokerId
diff --git a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala
index 6ce6e9e0a48de..6bdc64880896e 100644
--- a/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala
+++ b/core/src/main/scala/kafka/server/NodeToControllerChannelManager.scala
@@ -59,8 +59,8 @@ class MetadataCacheControllerNodeProvider(
val quorumControllerNodeProvider: () => Option[ControllerInformation]
) extends ControllerNodeProvider {
- private val zkControllerListenerName = config.controlPlaneListenerName.getOrElse(config.interBrokerListenerName)
- private val zkControllerSecurityProtocol = config.controlPlaneSecurityProtocol.getOrElse(config.interBrokerSecurityProtocol)
+ private val zkControllerListenerName = config.controlPlaneListenerName.orElse(config.interBrokerListenerName)
+ private val zkControllerSecurityProtocol = config.controlPlaneSecurityProtocol.orElse(config.interBrokerSecurityProtocol)
private val zkControllerSaslMechanism = config.saslMechanismInterBrokerProtocol
val emptyZkControllerInfo = ControllerInformation(
@@ -89,8 +89,8 @@ object RaftControllerNodeProvider {
raftManager: RaftManager[ApiMessageAndVersion],
config: KafkaConfig,
): RaftControllerNodeProvider = {
- val controllerListenerName = new ListenerName(config.controllerListenerNames.head)
- val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value()))
+ val controllerListenerName = new ListenerName(config.controllerListenerNames.asScala.head)
+ val controllerSecurityProtocol = config.effectiveListenerSecurityProtocolMap.asScala.getOrElse(controllerListenerName, SecurityProtocol.forName(controllerListenerName.value()))
val controllerSaslMechanism = config.saslMechanismControllerProtocol
new RaftControllerNodeProvider(
raftManager,
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index 6ab67a84ec426..693c55728881d 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -1240,7 +1240,7 @@ class ReplicaManager(val config: KafkaConfig,
def describeLogDirs(partitions: Set[TopicPartition]): List[DescribeLogDirsResponseData.DescribeLogDirsResult] = {
val logsByDir = logManager.allLogs.groupBy(log => log.parentDir)
- config.logDirs.toSet.map { logDir: String =>
+ config.logDirs.asScala.toSet.map { logDir: String =>
val file = Paths.get(logDir)
val absolutePath = file.toAbsolutePath.toString
try {
@@ -2487,7 +2487,7 @@ class ReplicaManager(val config: KafkaConfig,
s"for partitions ${partitionsWithOfflineFutureReplica.mkString(",")} because they are in the failed log directory $dir.")
}
logManager.handleLogDirFailure(dir)
- if (dir == new File(config.metadataLogDir).getAbsolutePath && (config.processRoles.nonEmpty || config.migrationEnabled)) {
+ if (dir == new File(config.metadataLogDir).getAbsolutePath && (!config.processRoles.isEmpty || config.migrationEnabled)) {
fatal(s"Shutdown broker because the metadata log dir $dir has failed")
Exit.halt(1)
}
@@ -2557,7 +2557,7 @@ class ReplicaManager(val config: KafkaConfig,
}
private def createReplicaSelector(): Option[ReplicaSelector] = {
- config.replicaSelectorClassName.map { className =>
+ config.replicaSelectorClassName.asScala.map { className =>
val tmpReplicaSelector: ReplicaSelector = CoreUtils.createObject[ReplicaSelector](className)
tmpReplicaSelector.configure(config.originals())
tmpReplicaSelector
diff --git a/core/src/main/scala/kafka/tools/StorageTool.scala b/core/src/main/scala/kafka/tools/StorageTool.scala
index c9f5139585225..3df85a65d195e 100644
--- a/core/src/main/scala/kafka/tools/StorageTool.scala
+++ b/core/src/main/scala/kafka/tools/StorageTool.scala
@@ -152,7 +152,7 @@ object StorageTool extends Logging {
}
}
try {
- config.validateWithMetadataVersion(metadataVersion)
+ config.validator().validateWithMetadataVersion(metadataVersion)
} catch {
case e: IllegalArgumentException => throw new TerseFailure(s"Invalid configuration for metadata version: ${e.getMessage}")
}
@@ -233,12 +233,12 @@ object StorageTool extends Logging {
def configToLogDirectories(config: KafkaConfig): Seq[String] = {
val directories = new mutable.TreeSet[String]
- directories ++= config.logDirs
+ directories ++= config.logDirs.asScala
Option(config.metadataLogDir).foreach(directories.add)
directories.toSeq
}
- private def configToSelfManagedMode(config: KafkaConfig): Boolean = config.processRoles.nonEmpty
+ private def configToSelfManagedMode(config: KafkaConfig): Boolean = !config.processRoles.isEmpty
def getMetadataVersion(
namespace: Namespace,
diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala
index 8da7a4e7cc147..be1ac7a0587ec 100755
--- a/core/src/main/scala/kafka/utils/CoreUtils.scala
+++ b/core/src/main/scala/kafka/utils/CoreUtils.scala
@@ -26,7 +26,6 @@ import com.typesafe.scalalogging.Logger
import javax.management._
import scala.collection._
-import scala.collection.Seq
import kafka.cluster.EndPoint
import org.apache.commons.validator.routines.InetAddressValidator
import org.apache.kafka.common.network.ListenerName
@@ -81,7 +80,7 @@ object CoreUtils {
* Recursively delete the list of files/directories and any subfiles (if any exist)
* @param files sequence of files to be deleted
*/
- def delete(files: Seq[String]): Unit = files.foreach(f => Utils.delete(new File(f)))
+ def delete(files: util.List[String]): Unit = files.forEach(f => Utils.delete(new File(f)))
/**
* Register the given mbean with the platform mbean server,
diff --git a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java
index 168c3298343db..054f28f8e2c3f 100644
--- a/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java
+++ b/core/src/test/java/kafka/server/MetadataVersionConfigValidatorTest.java
@@ -24,6 +24,7 @@
import org.apache.kafka.image.loader.LogDeltaManifest;
import org.apache.kafka.raft.LeaderAndEpoch;
import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.config.KafkaConfigValidator;
import org.apache.kafka.server.fault.FaultHandler;
import org.junit.jupiter.api.Test;
@@ -69,12 +70,13 @@ void testValidatesConfigOnMetadataChange() throws Exception {
MetadataVersion metadataVersion = MetadataVersion.IBP_3_7_IV2;
KafkaConfig config = mock(KafkaConfig.class);
FaultHandler faultHandler = mock(FaultHandler.class);
+ KafkaConfigValidator validator = mock(KafkaConfigValidator.class);
when(config.brokerId()).thenReturn(8);
-
+ when(config.validator()).thenReturn(validator);
testWith(metadataVersion, config, faultHandler);
- verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion));
+ verify(validator, times(1)).validateWithMetadataVersion(eq(metadataVersion));
verifyNoMoreInteractions(faultHandler);
}
@@ -85,16 +87,18 @@ void testInvokesFaultHandlerOnException() throws Exception {
Exception exception = new Exception();
KafkaConfig config = mock(KafkaConfig.class);
FaultHandler faultHandler = mock(FaultHandler.class);
+ KafkaConfigValidator validator = mock(KafkaConfigValidator.class);
+ when(config.validator()).thenReturn(validator);
when(faultHandler.handleFault(any(), any())).thenReturn(new RuntimeException("returned exception"));
when(config.brokerId()).thenReturn(8);
willAnswer(invocation -> {
throw exception;
- }).given(config).validateWithMetadataVersion(eq(metadataVersion));
+ }).given(validator).validateWithMetadataVersion(eq(metadataVersion));
testWith(metadataVersion, config, faultHandler);
- verify(config, times(1)).validateWithMetadataVersion(eq(metadataVersion));
+ verify(validator, times(1)).validateWithMetadataVersion(eq(metadataVersion));
verify(faultHandler, times(1)).handleFault(
eq("Broker configuration does not support the cluster MetadataVersion"),
eq(exception));
diff --git a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
index 3a7bac5aad5ea..9b66bf95751ff 100644
--- a/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/RaftClusterInvocationContext.java
@@ -39,7 +39,6 @@
import org.junit.jupiter.api.extension.BeforeTestExecutionCallback;
import org.junit.jupiter.api.extension.Extension;
import org.junit.jupiter.api.extension.TestTemplateInvocationContext;
-import scala.compat.java8.OptionConverters;
import java.util.ArrayList;
import java.util.Arrays;
@@ -134,7 +133,7 @@ public ListenerName clientListener() {
public Optional controllerListenerName() {
return controllers().values().stream()
.findAny()
- .flatMap(s -> OptionConverters.toJava(s.config().controllerListenerNames().headOption()))
+ .flatMap(s -> Optional.ofNullable(s.config().controllerListenerNames().get(0)))
.map(ListenerName::new);
}
diff --git a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
index 4eecfa7bf2141..7df8bd097bdfa 100644
--- a/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
+++ b/core/src/test/java/kafka/test/junit/ZkClusterInvocationContext.java
@@ -137,7 +137,7 @@ public ListenerName clientListener() {
@Override
public Optional controlPlaneListenerName() {
- return OptionConverters.toJava(clusterReference.get().servers().head().config().controlPlaneListenerName());
+ return clusterReference.get().servers().head().config().controlPlaneListenerName();
}
@Override
diff --git a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
index 342ff3ab8e7fc..58d1dc279517e 100644
--- a/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/admin/ListOffsetsIntegrationTest.scala
@@ -258,7 +258,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
// case 2: test the offsets from recovery path.
// server will rebuild offset index according to log files if the index files are nonexistent
- val indexFiles = brokers.flatMap(_.config.logDirs).toSet
+ val indexFiles = brokers.flatMap(_.config.logDirs.asScala).toSet
brokers.foreach(b => killBroker(b.config.brokerId))
indexFiles.foreach { root =>
val files = new File(s"$root/$topic-0").listFiles()
diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
index 194186968b12d..13aa3368a114f 100644
--- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala
@@ -307,11 +307,11 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
// Generate two mutually exclusive replicaAssignment
val firstReplicaAssignment = brokers.map { server =>
- val logDir = new File(server.config.logDirs(randomNums(server))).getAbsolutePath
+ val logDir = new File(server.config.logDirs.get(randomNums(server))).getAbsolutePath
new TopicPartitionReplica(topic, 0, server.config.brokerId) -> logDir
}.toMap
val secondReplicaAssignment = brokers.map { server =>
- val logDir = new File(server.config.logDirs(1 - randomNums(server))).getAbsolutePath
+ val logDir = new File(server.config.logDirs.get(1 - randomNums(server))).getAbsolutePath
new TopicPartitionReplica(topic, 0, server.config.brokerId) -> logDir
}.toMap
@@ -900,7 +900,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}
// we will create another dir just for one server
- val futureLogDir = brokers(0).config.logDirs(1)
+ val futureLogDir = brokers(0).config.logDirs.get(1)
val futureReplica = new TopicPartitionReplica(topic, 0, brokers(0).config.brokerId)
// Verify that replica can be moved to the specified log directory
@@ -2216,7 +2216,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertTrue(results.containsKey(invalidTopicName))
assertFutureExceptionTypeEquals(results.get(invalidTopicName), classOf[InvalidTopicException])
assertFutureExceptionTypeEquals(client.alterReplicaLogDirs(
- Map(new TopicPartitionReplica(longTopicName, 0, 0) -> brokers(0).config.logDirs(0)).asJava).all(),
+ Map(new TopicPartitionReplica(longTopicName, 0, 0) -> brokers(0).config.logDirs.get(0)).asJava).all(),
classOf[InvalidTopicException])
client.close()
}
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index 8508607eefde5..eb814e73c744e 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -24,12 +24,13 @@ import java.lang.management.ManagementFactory
import java.security.KeyStore
import java.time.Duration
import java.util
-import java.util.{Collections, Properties}
+import java.util.{Collections, Optional, Properties}
import java.util.concurrent._
import javax.management.ObjectName
import com.yammer.metrics.core.MetricName
import kafka.admin.ConfigCommand
import kafka.api.{KafkaSasl, SaslSetup}
+import kafka.cluster.EndPoint
import kafka.controller.{ControllerBrokerStateInfo, ControllerChannelManager}
import kafka.log.UnifiedLog
import kafka.network.{DataPlaneAcceptor, Processor, RequestChannel}
@@ -80,6 +81,7 @@ import scala.collection._
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
import scala.collection.Seq
+import scala.compat.java8.OptionConverters.RichOptionalGeneric
object DynamicBrokerReconfigurationTest {
val Plain = "PLAIN"
@@ -1121,7 +1123,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val passwordConfigs = props.asScala.filter { case (k, _) => DynamicBrokerConfig.isPasswordConfig(k) }
assertTrue(passwordConfigs.nonEmpty, "Password configs not found")
val passwordDecoder = createPasswordEncoder(config, config.passwordEncoderSecret)
- val passwordEncoder = createPasswordEncoder(config, Some(new Password(oldSecret)))
+ val passwordEncoder = createPasswordEncoder(config, Optional.of(new Password(oldSecret)))
passwordConfigs.foreach { case (name, value) =>
val decoded = passwordDecoder.decode(value).value
propsEncodedWithOldSecret.put(name, passwordEncoder.encode(new Password(decoded)))
@@ -1211,10 +1213,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
saslMechanisms: Seq[String]): Unit = {
val config = servers.head.config
val existingListenerCount = config.listeners.size
- val listeners = config.listeners
+ val listeners = config.listeners.asScala
+ .map(EndPoint.fromJava)
.map(e => s"${e.listenerName.value}://${e.host}:${e.port}")
.mkString(",") + s",$listenerName://localhost:0"
- val listenerMap = config.effectiveListenerSecurityProtocolMap
+ val listenerMap = config.effectiveListenerSecurityProtocolMap.asScala
.map { case (name, protocol) => s"${name.value}:${protocol.name}" }
.mkString(",") + s",$listenerName:${securityProtocol.name}"
@@ -1321,11 +1324,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val config = servers.head.config
val existingListenerCount = config.listeners.size
- val listeners = config.listeners
+ val listeners = config.listeners.asScala
+ .map(EndPoint.fromJava)
.filter(e => e.listenerName.value != securityProtocol.name)
.map(e => s"${e.listenerName.value}://${e.host}:${e.port}")
.mkString(",")
- val listenerMap = config.effectiveListenerSecurityProtocolMap
+ val listenerMap = config.effectiveListenerSecurityProtocolMap.asScala
.filter { case (listenerName, _) => listenerName.value != securityProtocol.name }
.map { case (listenerName, protocol) => s"${listenerName.value}:${protocol.name}" }
.mkString(",")
@@ -1515,11 +1519,12 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
private def alterAdvertisedListener(adminClient: Admin, externalAdminClient: Admin, oldHost: String, newHost: String): Unit = {
val configs = servers.map { server =>
val resource = new ConfigResource(ConfigResource.Type.BROKER, server.config.brokerId.toString)
- val newListeners = server.config.effectiveAdvertisedListeners.map { e =>
- if (e.listenerName.value == SecureExternal)
- s"${e.listenerName.value}://$newHost:${server.boundPort(e.listenerName)}"
+ val newListeners = server.config.effectiveAdvertisedListeners.asScala.map { e =>
+ val endPoint = EndPoint.fromJava(e)
+ if (endPoint.listenerName.value == SecureExternal)
+ s"${endPoint.listenerName.value}://$newHost:${server.boundPort(endPoint.listenerName)}"
else
- s"${e.listenerName.value}://${e.host}:${server.boundPort(e.listenerName)}"
+ s"${endPoint.listenerName.value}://${e.host}:${server.boundPort(endPoint.listenerName)}"
}.mkString(",")
val configEntry = new ConfigEntry(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, newListeners)
(resource, new Config(Collections.singleton(configEntry)))
@@ -1527,7 +1532,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
adminClient.alterConfigs(configs).all.get
servers.foreach { server =>
TestUtils.retry(10000) {
- val externalListener = server.config.effectiveAdvertisedListeners.find(_.listenerName.value == SecureExternal)
+ val externalListener = server.config.effectiveAdvertisedListeners.asScala.find(EndPoint.fromJava(_).listenerName.value == SecureExternal)
.getOrElse(throw new IllegalStateException("External listener not found"))
assertEquals(newHost, externalListener.host, "Config not updated")
}
@@ -1595,7 +1600,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val externalListenerPrefix = listenerPrefix(SecureExternal)
val sslStoreProps = new Properties
sslStoreProps ++= securityProps(sslProperties, KEYSTORE_PROPS, externalListenerPrefix)
- sslStoreProps.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, kafkaConfig.passwordEncoderSecret.map(_.value).orNull)
+ sslStoreProps.put(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG, kafkaConfig.passwordEncoderSecret.map(_.value).asScala.orNull)
zkClient.makeSurePersistentPathExists(ConfigEntityChangeNotificationZNode.path)
val entityType = ConfigType.BROKER
@@ -1625,8 +1630,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
passwordEncoder.decode(brokerProps.getProperty(s"$externalListenerPrefix$SSL_KEY_PASSWORD_CONFIG")))
}
- private def createPasswordEncoder(config: KafkaConfig, secret: Option[Password]): PasswordEncoder = {
- val encoderSecret = secret.getOrElse(throw new IllegalStateException("Password encoder secret not configured"))
+ private def createPasswordEncoder(config: KafkaConfig, secret: Optional[Password]): PasswordEncoder = {
+ val encoderSecret = secret.orElseThrow(() => new IllegalStateException("Password encoder secret not configured"))
PasswordEncoder.encrypting(encoderSecret,
config.passwordEncoderKeyFactoryAlgorithm,
config.passwordEncoderCipherAlgorithm,
diff --git a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
index c9f3c46f0d8c2..c822a05f8d731 100644
--- a/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/FetchFromFollowerIntegrationTest.scala
@@ -197,7 +197,7 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RangeAssignor].getName)
val consumers = brokers.map { server =>
consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
- consumerConfig.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, server.config.rack.orNull)
+ consumerConfig.setProperty(ConsumerConfig.CLIENT_RACK_CONFIG, server.config.rack.orElse(null))
consumerConfig.setProperty(ConsumerConfig.GROUP_INSTANCE_ID_CONFIG, s"instance-${server.config.brokerId}")
consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "1000")
createConsumer()
diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
index 8be2b994116f5..949d3afe9e389 100644
--- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
+++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala
@@ -17,6 +17,7 @@
package kafka.server
+import kafka.cluster.EndPoint
import kafka.log.UnifiedLog
import kafka.network.SocketServer
import kafka.server.IntegrationTestUtils.connectAndReceive
@@ -115,7 +116,7 @@ class KRaftClusterTest {
cluster.format()
cluster.startup()
val controller = cluster.controllers().values().iterator().asScala.filter(_.controller.isActive).next()
- val port = controller.socketServer.boundPort(controller.config.controllerListeners.head.listenerName)
+ val port = controller.socketServer.boundPort(EndPoint.fromJava(controller.config.controllerListeners.asScala.head).listenerName)
// shutdown active controller
controller.shutdown()
@@ -1555,7 +1556,7 @@ class KRaftClusterTest {
// Copy foo-0 to targetParentDir
// This is so that we can rename the main replica to a future down below
val parentDir = log.parentDir
- val targetParentDir = broker0.config.logDirs.filter(_ != parentDir).head
+ val targetParentDir = broker0.config.logDirs.asScala.filter(_ != parentDir).head
val targetDirFile = new File(targetParentDir, log.dir.getName)
FileUtils.copyDirectory(log.dir, targetDirFile)
assertTrue(targetDirFile.exists())
diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala
index fed241f5d7b09..9c85caea580d6 100644
--- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala
+++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithSameSecurityProtocolBaseTest.scala
@@ -21,6 +21,7 @@ package kafka.server
import java.util.{Collections, Objects, Properties}
import java.util.concurrent.TimeUnit
import kafka.api.SaslSetup
+import kafka.cluster.EndPoint
import kafka.utils.JaasTestUtils.JaasSection
import kafka.utils.{JaasTestUtils, TestUtils}
import kafka.utils.Implicits._
@@ -115,7 +116,8 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
createScramCredentials(zkConnect, JaasTestUtils.KafkaScramUser, JaasTestUtils.KafkaScramPassword)
- servers.head.config.listeners.foreach { endPoint =>
+ servers.head.config.listeners.forEach { e =>
+ val endPoint = EndPoint.fromJava(e)
val listenerName = endPoint.listenerName
val trustStoreFile =
diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
index ac6a8bcbd4111..b85548e5659a5 100755
--- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
+++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala
@@ -103,7 +103,7 @@ class KRaftQuorumImplementation(
): KafkaBroker = {
val metaPropertiesEnsemble = {
val loader = new MetaPropertiesEnsemble.Loader()
- config.logDirs.foreach(loader.addLogDir)
+ config.logDirs.forEach(loader.addLogDir)
loader.addMetadataLogDir(config.metadataLogDir)
val ensemble = loader.load()
val copier = new MetaPropertiesEnsemble.Copier(ensemble)
diff --git a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
index e98a8fdeccb70..9587015201282 100644
--- a/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/zk/ZkMigrationIntegrationTest.scala
@@ -60,6 +60,7 @@ import java.util
import java.util.concurrent.{CompletableFuture, ExecutionException, TimeUnit}
import java.util.{Collections, Optional, UUID}
import scala.collection.Seq
+import scala.compat.java8.OptionConverters.RichOptionalGeneric
import scala.jdk.CollectionConverters._
object ZkMigrationIntegrationTest {
@@ -248,7 +249,7 @@ class ZkMigrationIntegrationTest {
val zkClient = clusterInstance.asInstanceOf[ZkClusterInstance].getUnderlying().zkClient
val kafkaConfig = clusterInstance.asInstanceOf[ZkClusterInstance].getUnderlying.servers.head.config
- val zkConfigEncoder = kafkaConfig.passwordEncoderSecret match {
+ val zkConfigEncoder = kafkaConfig.passwordEncoderSecret.asScala match {
case Some(secret) =>
PasswordEncoder.encrypting(secret,
kafkaConfig.passwordEncoderKeyFactoryAlgorithm,
diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
index 9a5483ee5b667..4f8823e6d1d7a 100644
--- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
+++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
@@ -58,7 +58,7 @@ class LocalLeaderEndPointTest extends Logging {
def setUp(): Unit = {
val props = TestUtils.createBrokerConfig(sourceBroker.id, TestUtils.MockZkConnect, port = sourceBroker.port)
val config = KafkaConfig.fromProps(props)
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val alterPartitionManager = mock(classOf[AlterPartitionManager])
val metrics = new Metrics
quotaManager = QuotaFactory.instantiate(config, metrics, time, "")
diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
index bd1ee39a4835e..de1ab1b10ea98 100644
--- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
@@ -426,7 +426,7 @@ class LogConfigTest {
def validate(metadataVersion: MetadataVersion, jbodConfig: Boolean): Unit =
KafkaConfig.fromProps(
TestUtils.createBrokerConfig(nodeId = 0, zkConnect = null, logDirCount = if (jbodConfig) 2 else 1)
- ).validateWithMetadataVersion(metadataVersion)
+ ).validator().validateWithMetadataVersion(metadataVersion)
validate(MetadataVersion.IBP_3_6_IV2, jbodConfig = false)
validate(MetadataVersion.IBP_3_7_IV0, jbodConfig = false)
diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
index dce053489f0d5..db2acbd09afeb 100644
--- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
+++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala
@@ -87,7 +87,7 @@ class SocketServerTest {
private val kafkaLogger = org.apache.log4j.LogManager.getLogger("kafka")
private var logLevelToRestore: Level = _
def endpoint: EndPoint = {
- KafkaConfig.fromProps(props, doLog = false).dataPlaneListeners.head
+ EndPoint.fromJava(KafkaConfig.fromProps(props, doLog = false).dataPlaneListeners.asScala.head)
}
def listener: String = endpoint.listenerName.value
val uncaughtExceptions = new AtomicInteger(0)
@@ -367,7 +367,8 @@ class SocketServerTest {
val config = KafkaConfig.fromProps(testProps)
val testableServer = new TestableSocketServer(config)
- val updatedEndPoints = config.effectiveAdvertisedListeners.map { endpoint =>
+ val updatedEndPoints = config.effectiveAdvertisedListeners.asScala.map { e =>
+ val endpoint = EndPoint.fromJava(e)
endpoint.copy(port = testableServer.boundPort(endpoint.listenerName))
}.map(_.toJava)
@@ -2221,7 +2222,7 @@ class SocketServerTest {
testableAcceptor.processors(0).asInstanceOf[TestableProcessor]
def testableAcceptor: TestableAcceptor = {
- val endpoint = this.config.dataPlaneListeners.head
+ val endpoint = EndPoint.fromJava(this.config.dataPlaneListeners.asScala.head)
dataPlaneAcceptors.get(endpoint).asInstanceOf[TestableAcceptor]
}
diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
index 4b3e170191506..3b7e90cab26c7 100644
--- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala
@@ -56,7 +56,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
val partitionNum = 5
// Alter replica dir before topic creation
- val logDir1 = new File(servers.head.config.logDirs(Random.nextInt(logDirCount))).getAbsolutePath
+ val logDir1 = new File(servers.head.config.logDirs.get(Random.nextInt(logDirCount))).getAbsolutePath
val partitionDirs1 = (0 until partitionNum).map(partition => new TopicPartition(topic, partition) -> logDir1).toMap
val alterReplicaLogDirsResponse1 = sendAlterReplicaLogDirsRequest(partitionDirs1)
@@ -73,7 +73,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
}
// Alter replica dir again after topic creation
- val logDir2 = new File(servers.head.config.logDirs(Random.nextInt(logDirCount))).getAbsolutePath
+ val logDir2 = new File(servers.head.config.logDirs.get(Random.nextInt(logDirCount))).getAbsolutePath
val partitionDirs2 = (0 until partitionNum).map(partition => new TopicPartition(topic, partition) -> logDir2).toMap
val alterReplicaLogDirsResponse2 = sendAlterReplicaLogDirsRequest(partitionDirs2)
// The response should succeed for all partitions
@@ -88,10 +88,10 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
@Test
def testAlterReplicaLogDirsRequestErrorCode(): Unit = {
- val offlineDir = new File(servers.head.config.logDirs.tail.head).getAbsolutePath
- val validDir1 = new File(servers.head.config.logDirs(1)).getAbsolutePath
- val validDir2 = new File(servers.head.config.logDirs(2)).getAbsolutePath
- val validDir3 = new File(servers.head.config.logDirs(3)).getAbsolutePath
+ val offlineDir = new File(servers.head.config.logDirs.asScala.tail.head).getAbsolutePath
+ val validDir1 = new File(servers.head.config.logDirs.get(1)).getAbsolutePath
+ val validDir2 = new File(servers.head.config.logDirs.get(2)).getAbsolutePath
+ val validDir3 = new File(servers.head.config.logDirs.get(3)).getAbsolutePath
// Test AlterReplicaDirRequest before topic creation
val partitionDirs1 = mutable.Map.empty[TopicPartition, String]
@@ -129,7 +129,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
val partitionNum = 1
// Alter replica dir before topic creation
- val logDir1 = new File(servers.head.config.logDirs(1)).getAbsolutePath
+ val logDir1 = new File(servers.head.config.logDirs.get(1)).getAbsolutePath
val partitionDirs1 = (0 until partitionNum).map(partition => new TopicPartition(topic, partition) -> logDir1).toMap
val alterReplicaLogDirsResponse1 = sendAlterReplicaLogDirsRequest(partitionDirs1)
@@ -162,7 +162,7 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
}, "timed out waiting for log segment to retention")
// Alter replica dir again after topic creation
- val logDir2 = new File(servers.head.config.logDirs(2)).getAbsolutePath
+ val logDir2 = new File(servers.head.config.logDirs.get(2)).getAbsolutePath
val alterReplicaLogDirsResponse2 = sendAlterReplicaLogDirsRequest(Map(tp -> logDir2))
// The response should succeed for all partitions
assertEquals(Errors.NONE, findErrorForPartition(alterReplicaLogDirsResponse2, tp))
diff --git a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
index c33622a0038e4..176dc78d663e4 100644
--- a/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ControllerRegistrationManagerTest.scala
@@ -39,7 +39,6 @@ import org.junit.jupiter.params.provider.ValueSource
import java.util
import java.util.{OptionalInt, Properties}
import java.util.concurrent.{CompletableFuture, TimeUnit}
-import scala.jdk.CollectionConverters._
@Timeout(value = 60)
class ControllerRegistrationManagerTest {
@@ -77,7 +76,7 @@ class ControllerRegistrationManagerTest {
createSupportedFeatures(MetadataVersion.IBP_3_7_IV0),
false,
RecordTestUtils.createTestControllerRegistration(1, false).incarnationId(),
- ListenerInfo.create(context.config.controllerListeners.map(_.toJava).asJava),
+ ListenerInfo.create(context.config.controllerListeners),
new ExponentialBackoff(1, 2, 100, 0.02))
}
diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
index 0148d29061e80..64f284e29e688 100644
--- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala
@@ -66,7 +66,7 @@ class DescribeClusterRequestTest extends BaseRequestTest {
.setBrokerId(server.config.brokerId)
.setHost("localhost")
.setPort(server.socketServer.boundPort(listenerName))
- .setRack(server.config.rack.orNull)
+ .setRack(server.config.rack.orElse(null))
}.toSet
var expectedControllerId = 0
diff --git a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala
index 5a0a5d8dc180a..1b563c782544b 100644
--- a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala
@@ -40,8 +40,9 @@ class DescribeLogDirsRequestTest extends BaseRequestTest {
@Test
def testDescribeLogDirsRequest(): Unit = {
- val onlineDir = new File(servers.head.config.logDirs.head).getAbsolutePath
- val offlineDir = new File(servers.head.config.logDirs.tail.head).getAbsolutePath
+ val logDirs = servers.head.config.logDirs.asScala
+ val onlineDir = new File(logDirs.head).getAbsolutePath
+ val offlineDir = new File(servers.head.config.logDirs.asScala.tail.head).getAbsolutePath
servers.head.replicaManager.handleLogDirFailure(offlineDir)
createTopic(topic, partitionNum, 1)
TestUtils.generateAndProduceMessages(servers, topic, 10)
diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
index 170e2e417ca60..f9f669de671bf 100755
--- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala
@@ -30,6 +30,8 @@ import org.apache.kafka.common.record.SimpleRecord
import org.apache.kafka.server.util.{KafkaScheduler, MockTime}
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogDirFailureChannel}
+import scala.jdk.CollectionConverters.CollectionHasAsScala
+
class HighwatermarkPersistenceTest {
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps)
@@ -37,7 +39,7 @@ class HighwatermarkPersistenceTest {
val configRepository = new MockConfigRepository()
val logManagers = configs map { config =>
TestUtils.createLogManager(
- logDirs = config.logDirs.map(new File(_)),
+ logDirs = config.logDirs.asScala.map(new File(_)).toBuffer,
cleanerConfig = new CleanerConfig(true))
}
@@ -191,7 +193,7 @@ class HighwatermarkPersistenceTest {
}
private def hwmFor(replicaManager: ReplicaManager, topic: String, partition: Int): Long = {
- replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs.head).getAbsolutePath).read().getOrElse(
+ replicaManager.highWatermarkCheckpoints(new File(replicaManager.config.logDirs.asScala.head).getAbsolutePath).read().getOrElse(
new TopicPartition(topic, partition), 0L)
}
}
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index 506925c3339cb..4bce5b407ae62 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -32,6 +32,7 @@ import org.apache.kafka.common.record.{CompressionType, Records}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
+import org.apache.kafka.common.utils.Utils
import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
import org.apache.kafka.coordinator.group.Group.GroupType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
@@ -49,6 +50,7 @@ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.api.function.Executable
+import java.util.stream.Collectors
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._
@@ -165,7 +167,7 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, s"PLAINTEXT://$hostName:$port")
val serverConfig = KafkaConfig.fromProps(props)
- val endpoints = serverConfig.effectiveAdvertisedListeners
+ val endpoints = serverConfig.effectiveAdvertisedListeners.asScala
assertEquals(1, endpoints.size)
val endpoint = endpoints.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get
assertEquals(endpoint.host, hostName)
@@ -181,7 +183,7 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, s"PLAINTEXT://$advertisedHostName:$advertisedPort")
val serverConfig = KafkaConfig.fromProps(props)
- val endpoints = serverConfig.effectiveAdvertisedListeners
+ val endpoints = serverConfig.effectiveAdvertisedListeners.asScala
val endpoint = endpoints.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get
assertEquals(endpoint.host, advertisedHostName)
@@ -274,14 +276,14 @@ class KafkaConfigTest {
assertEquals(SecurityProtocol.SSL, controlEndpoint.securityProtocol)
//advertised listener should contain control-plane listener
- val advertisedEndpoints = serverConfig.effectiveAdvertisedListeners
+ val advertisedEndpoints = serverConfig.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava)
assertTrue(advertisedEndpoints.exists { endpoint =>
- endpoint.securityProtocol == controlEndpoint.securityProtocol && endpoint.listenerName.value().equals(controlEndpoint.listenerName.value())
+ endpoint.securityProtocol == controlEndpoint.securityProtocol && endpoint.listenerName.value().equals(controlEndpoint.listenerName.get())
})
// interBrokerListener name should be different from control-plane listener name
val interBrokerListenerName = serverConfig.interBrokerListenerName
- assertFalse(interBrokerListenerName.value().equals(controlEndpoint.listenerName.value()))
+ assertFalse(interBrokerListenerName.value().equals(controlEndpoint.listenerName.get()))
}
@Test
@@ -414,7 +416,7 @@ class KafkaConfigTest {
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),
+ assertEquals(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(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "CONTROLLER,SSL")
@@ -427,7 +429,7 @@ class KafkaConfigTest {
props.remove(SocketServerConfigs.LISTENERS_CONFIG)
// ensure we don't map it to PLAINTEXT when it is explicitly mapped otherwise
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,CONTROLLER:SSL")
- assertEquals(Some(SecurityProtocol.SSL),
+ assertEquals(SecurityProtocol.SSL,
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(controllerListenerName))
// ensure we don't map it to PLAINTEXT when anything is explicitly given
// (i.e. it is only part of the default value, even with KRaft)
@@ -436,7 +438,7 @@ class KafkaConfigTest {
// 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(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, "SSL")
- assertEquals(Some(SecurityProtocol.SSL),
+ assertEquals(SecurityProtocol.SSL,
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("SSL")))
}
@@ -448,9 +450,9 @@ class KafkaConfigTest {
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),
+ assertEquals(SecurityProtocol.PLAINTEXT,
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER1")))
- assertEquals(Some(SecurityProtocol.PLAINTEXT),
+ assertEquals(SecurityProtocol.PLAINTEXT,
KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER2")))
}
@@ -464,7 +466,7 @@ class KafkaConfigTest {
"Error creating broker listeners from 'CONTROLLER://localhost:9092': No security protocol defined for listener CONTROLLER")
// Valid now
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "PLAINTEXT://localhost:9092")
- assertEquals(None, KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER")))
+ assertTrue(Option(KafkaConfig.fromProps(props).effectiveListenerSecurityProtocolMap.get(new ListenerName("CONTROLLER"))).isEmpty)
}
@Test
@@ -491,12 +493,12 @@ class KafkaConfigTest {
EndPoint("localhost", 9091, new ListenerName("CLIENT"), SecurityProtocol.SSL),
EndPoint("localhost", 9092, new ListenerName("REPLICATION"), SecurityProtocol.SSL),
EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT))
- assertEquals(expectedListeners, config.listeners)
- assertEquals(expectedListeners, config.effectiveAdvertisedListeners)
- val expectedSecurityProtocolMap = Map(
- new ListenerName("CLIENT") -> SecurityProtocol.SSL,
- new ListenerName("REPLICATION") -> SecurityProtocol.SSL,
- new ListenerName("INTERNAL") -> SecurityProtocol.PLAINTEXT
+ assertEquals(expectedListeners, config.listeners.asScala.map(EndPoint.fromJava))
+ assertEquals(expectedListeners, config.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava))
+ val expectedSecurityProtocolMap = Utils.mkMap(
+ Utils.mkEntry(new ListenerName("CLIENT"), SecurityProtocol.SSL),
+ Utils.mkEntry(new ListenerName("REPLICATION"), SecurityProtocol.SSL),
+ Utils.mkEntry(new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)
)
assertEquals(expectedSecurityProtocolMap, config.effectiveListenerSecurityProtocolMap)
}
@@ -517,19 +519,19 @@ class KafkaConfigTest {
EndPoint("localhost", 9091, new ListenerName("EXTERNAL"), SecurityProtocol.SSL),
EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)
)
- assertEquals(expectedListeners, config.listeners)
+ assertEquals(expectedListeners, config.listeners.asScala.map(EndPoint.fromJava))
val expectedAdvertisedListeners = Seq(
EndPoint("lb1.example.com", 9000, new ListenerName("EXTERNAL"), SecurityProtocol.SSL),
EndPoint("host1", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)
)
- assertEquals(expectedAdvertisedListeners, config.effectiveAdvertisedListeners)
+ assertEquals(expectedAdvertisedListeners, config.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava))
val expectedSecurityProtocolMap = Map(
new ListenerName("EXTERNAL") -> SecurityProtocol.SSL,
new ListenerName("INTERNAL") -> SecurityProtocol.PLAINTEXT
)
- assertEquals(expectedSecurityProtocolMap, config.effectiveListenerSecurityProtocolMap)
+ assertEquals(expectedSecurityProtocolMap, config.effectiveListenerSecurityProtocolMap.asScala)
}
@Test
@@ -573,8 +575,9 @@ class KafkaConfigTest {
props.setProperty(ZkConfigs.ZK_CONNECT_CONFIG, "localhost:2181")
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092")
val config = KafkaConfig.fromProps(props)
- assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listenerName.value == "SSL").map(_.connectionString))
- assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listenerName.value == "PLAINTEXT").map(_.connectionString))
+ val endPoints = config.listeners.asScala.map(EndPoint.fromJava)
+ assertEquals(Some("SSL://localhost:9092"), endPoints.find(_.listenerName.value == "SSL").map(_.connectionString))
+ assertEquals(Some("PLAINTEXT://localhost:9091"), endPoints.find(_.listenerName.value == "PLAINTEXT").map(_.connectionString))
}
private def listenerListToEndPoints(listenerList: String,
@@ -589,9 +592,10 @@ class KafkaConfigTest {
// configuration with no listeners
val conf = KafkaConfig.fromProps(props)
- assertEquals(listenerListToEndPoints("PLAINTEXT://:9092"), conf.listeners)
- assertNull(conf.listeners.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get.host)
- assertEquals(conf.effectiveAdvertisedListeners, listenerListToEndPoints("PLAINTEXT://:9092"))
+ val listeners = conf.listeners.asScala.map(EndPoint.fromJava)
+ assertEquals(listenerListToEndPoints("PLAINTEXT://:9092"), listeners)
+ assertNull(listeners.find(_.securityProtocol == SecurityProtocol.PLAINTEXT).get.host)
+ assertEquals(conf.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava), listenerListToEndPoints("PLAINTEXT://:9092"))
}
@nowarn("cat=deprecation")
@@ -1220,9 +1224,9 @@ class KafkaConfigTest {
assertEquals(false, config.brokerIdGenerationEnable)
assertEquals(1, config.maxReservedBrokerId)
assertEquals(1, config.brokerId)
- assertEquals(Seq("PLAINTEXT://127.0.0.1:1122"), config.effectiveAdvertisedListeners.map(_.connectionString))
- assertEquals(Map("127.0.0.1" -> 2, "127.0.0.2" -> 3), config.maxConnectionsPerIpOverrides)
- assertEquals(List("/tmp1", "/tmp2"), config.logDirs)
+ assertEquals(Seq("PLAINTEXT://127.0.0.1:1122"), config.effectiveAdvertisedListeners.asScala.map(EndPoint.fromJava(_).connectionString))
+ assertEquals(Utils.mkMap(Utils.mkEntry("127.0.0.1", 2), Utils.mkEntry("127.0.0.2", 3)), config.maxConnectionsPerIpOverrides)
+ assertEquals(Arrays.asList("/tmp1", "/tmp2"), config.logDirs)
assertEquals(12 * 60L * 1000L * 60, config.logRollTimeMillis)
assertEquals(11 * 60L * 1000L * 60, config.logRollTimeJitterMillis)
assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis)
@@ -1571,7 +1575,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
assertEquals(metadataDir, config.metadataLogDir)
- assertEquals(Seq(dataDir), config.logDirs)
+ assertEquals(Collections.singletonList(dataDir), config.logDirs)
}
@Test
@@ -1589,7 +1593,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
assertEquals(dataDir1, config.metadataLogDir)
- assertEquals(Seq(dataDir1, dataDir2), config.logDirs)
+ assertEquals(Arrays.asList(dataDir1, dataDir2), config.logDirs)
}
@Test
@@ -1707,7 +1711,11 @@ class KafkaConfigTest {
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()))
+ val earlyStartListeners: util.Set[String] = config.earlyStartListeners.stream()
+ .map(_.value())
+ .collect(Collectors.toSet())
+
+ assertEquals(Utils.mkSet("CONTROLLER"), earlyStartListeners)
}
@Test
@@ -1721,7 +1729,7 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG,
"INTERNAL://127.0.0.1:9092,INTERNAL2://127.0.0.1:9093")
val config = new KafkaConfig(props)
- assertEquals(Set(new ListenerName("INTERNAL"), new ListenerName("INTERNAL2")),
+ assertEquals(Utils.mkSet(new ListenerName("INTERNAL"), new ListenerName("INTERNAL2")),
config.earlyStartListeners)
}
diff --git a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
index a8b7a280defeb..80df6555824a1 100644
--- a/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogDirFailureTest.scala
@@ -116,7 +116,7 @@ class LogDirFailureTest extends IntegrationTestHarness {
props.put(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, "0.11.0")
props.put(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
val kafkaConfig = KafkaConfig.fromProps(props)
- val logDir = new File(kafkaConfig.logDirs.head)
+ val logDir = new File(kafkaConfig.logDirs.asScala.head)
// Make log directory of the partition on the leader broker inaccessible by replacing it with a file
CoreUtils.swallow(Utils.delete(logDir), this)
Files.createFile(logDir.toPath)
diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
index 21dba44f9f718..e596c79ed846a 100755
--- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala
@@ -34,6 +34,7 @@ import org.junit.jupiter.params.provider.ValueSource
import java.io.File
import java.util.Properties
import scala.collection.Seq
+import scala.jdk.CollectionConverters.ListHasAsScala
class LogRecoveryTest extends QuorumTestHarness {
@@ -62,8 +63,8 @@ class LogRecoveryTest extends QuorumTestHarness {
var admin: Admin = _
var producer: KafkaProducer[Integer, String] = _
- def hwFile1 = new OffsetCheckpointFile(new File(configProps1.logDirs.head, ReplicaManager.HighWatermarkFilename))
- def hwFile2 = new OffsetCheckpointFile(new File(configProps2.logDirs.head, ReplicaManager.HighWatermarkFilename))
+ def hwFile1 = new OffsetCheckpointFile(new File(configProps1.logDirs.asScala.head, ReplicaManager.HighWatermarkFilename))
+ def hwFile2 = new OffsetCheckpointFile(new File(configProps2.logDirs.asScala.head, ReplicaManager.HighWatermarkFilename))
var servers = Seq.empty[KafkaBroker]
// Some tests restart the brokers then produce more data. But since test brokers use random ports, we need
diff --git a/core/src/test/scala/unit/kafka/server/RegistrationTestContext.scala b/core/src/test/scala/unit/kafka/server/RegistrationTestContext.scala
index 92e318f4412cd..e86140472bfe0 100644
--- a/core/src/test/scala/unit/kafka/server/RegistrationTestContext.scala
+++ b/core/src/test/scala/unit/kafka/server/RegistrationTestContext.scala
@@ -64,11 +64,13 @@ class RegistrationTestContext(
val clusterId = "x4AJGXQSRnephtTZzujw4w"
val advertisedListeners = new ListenerCollection()
val controllerEpoch = new AtomicInteger(123)
- config.effectiveAdvertisedListeners.foreach { ep =>
- advertisedListeners.add(new Listener().setHost(ep.host).
- setName(ep.listenerName.value()).
+ config.effectiveAdvertisedListeners.stream().forEach { ep =>
+ val listener = new Listener().setHost(ep.host).
setPort(ep.port.shortValue()).
- setSecurityProtocol(ep.securityProtocol.id))
+ setSecurityProtocol(ep.securityProtocol.id)
+ ep.listenerName.map(listener.setName)
+
+ advertisedListeners.add(listener)
}
def poll(): Unit = {
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index 53b342e10db8a..69a722aa52898 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -154,7 +154,7 @@ class ReplicaManagerTest {
@Test
def testHighWaterMarkDirectoryMapping(): Unit = {
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val rm = new ReplicaManager(
metrics = metrics,
config = config,
@@ -170,7 +170,7 @@ class ReplicaManagerTest {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), None)
rm.checkpointHighWatermarks()
- config.logDirs.map(s => Paths.get(s, ReplicaManager.HighWatermarkFilename))
+ config.logDirs.asScala.map(s => Paths.get(s, ReplicaManager.HighWatermarkFilename))
.foreach(checkpointFile => assertTrue(Files.exists(checkpointFile),
s"checkpoint file does not exist at $checkpointFile"))
} finally {
@@ -183,7 +183,7 @@ class ReplicaManagerTest {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
val config = KafkaConfig.fromProps(props)
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val rm = new ReplicaManager(
metrics = metrics,
config = config,
@@ -199,7 +199,7 @@ class ReplicaManagerTest {
partition.createLogIfNotExists(isNew = false, isFutureReplica = false,
new LazyOffsetCheckpoints(rm.highWatermarkCheckpoints), None)
rm.checkpointHighWatermarks()
- config.logDirs.map(s => Paths.get(s, ReplicaManager.HighWatermarkFilename))
+ config.logDirs.asScala.map(s => Paths.get(s, ReplicaManager.HighWatermarkFilename))
.foreach(checkpointFile => assertTrue(Files.exists(checkpointFile),
s"checkpoint file does not exist at $checkpointFile"))
} finally {
@@ -209,7 +209,7 @@ class ReplicaManagerTest {
@Test
def testIllegalRequiredAcks(): Unit = {
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val rm = new ReplicaManager(
metrics = metrics,
config = config,
@@ -260,7 +260,7 @@ class ReplicaManagerTest {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect)
props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath)
val config = KafkaConfig.fromProps(props)
- val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties()))
+ val logManager = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), new LogConfig(new Properties()))
val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0)))
when(metadataCache.metadataVersion()).thenReturn(config.interBrokerProtocolVersion)
@@ -323,7 +323,7 @@ class ReplicaManagerTest {
val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect)
props.put("log.dirs", dir1.getAbsolutePath + "," + dir2.getAbsolutePath)
val config = KafkaConfig.fromProps(props)
- val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(new Properties()))
+ val logManager = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), new LogConfig(new Properties()))
val spyLogManager = spy(logManager)
val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, Seq(new Node(0, "host0", 0)))
@@ -396,7 +396,7 @@ class ReplicaManagerTest {
props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath)
val config = KafkaConfig.fromProps(props)
val logProps = new Properties()
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(logProps))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), new LogConfig(logProps))
val aliveBrokers = Seq(new Node(0, "host0", 0), new Node(1, "host1", 1))
val metadataCache: MetadataCache = mock(classOf[MetadataCache])
mockGetAliveBrokerFunctions(metadataCache, aliveBrokers)
@@ -2698,7 +2698,7 @@ class ReplicaManagerTest {
config = KafkaConfig.fromProps(props)
}
- val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), defaultConfig = new LogConfig(new Properties()), time = time)
+ val logManager = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), defaultConfig = new LogConfig(new Properties()), time = time)
val quotaManager = QuotaFactory.instantiate(config, metrics, time, "")
val replicaManager = new ReplicaManager(
metrics = metrics,
@@ -2782,7 +2782,7 @@ class ReplicaManagerTest {
@Test
def testUpdateStrayLogs(): Unit = {
- val logManager = TestUtils.createLogManager(config.logDirs.map(new File(_)), defaultConfig = new LogConfig(new Properties()), time = time)
+ val logManager = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), defaultConfig = new LogConfig(new Properties()), time = time)
val quotaManager = QuotaFactory.instantiate(config, metrics, time, "")
val replicaManager = new ReplicaManager(
metrics = metrics,
@@ -2902,7 +2902,7 @@ class ReplicaManagerTest {
props.asScala ++= extraProps.asScala
val config = KafkaConfig.fromProps(props)
val logConfig = new LogConfig(new Properties)
- val logDir = new File(new File(config.logDirs.head), s"$topic-$topicPartition")
+ val logDir = new File(new File(config.logDirs.asScala.head), s"$topic-$topicPartition")
Files.createDirectories(logDir.toPath)
val mockScheduler = new MockScheduler(time)
val mockBrokerTopicStats = new BrokerTopicStats
@@ -2959,7 +2959,7 @@ class ReplicaManagerTest {
// Expect to call LogManager.truncateTo exactly once
val topicPartitionObj = new TopicPartition(topic, topicPartition)
val mockLogMgr: LogManager = mock(classOf[LogManager])
- when(mockLogMgr.liveLogDirs).thenReturn(config.logDirs.map(new File(_).getAbsoluteFile))
+ when(mockLogMgr.liveLogDirs).thenReturn(config.logDirs.asScala.map(new File(_).getAbsoluteFile))
when(mockLogMgr.getOrCreateLog(ArgumentMatchers.eq(topicPartitionObj), ArgumentMatchers.eq(false), ArgumentMatchers.eq(false), any(), any())).thenReturn(mockLog)
when(mockLogMgr.getLog(topicPartitionObj, isFuture = false)).thenReturn(Some(mockLog))
when(mockLogMgr.getLog(topicPartitionObj, isFuture = true)).thenReturn(None)
@@ -3321,7 +3321,7 @@ class ReplicaManagerTest {
private def setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager: AddPartitionsToTxnManager,
transactionalTopicPartitions: List[TopicPartition],
config: KafkaConfig = config): ReplicaManager = {
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val metadataCache = mock(classOf[MetadataCache])
val replicaManager = new ReplicaManager(
@@ -3373,7 +3373,7 @@ class ReplicaManagerTest {
val mockLog = setupMockLog(path1)
if (setupLogDirMetaProperties) {
// add meta.properties file in each dir
- config.logDirs.foreach(dir => {
+ config.logDirs.forEach(dir => {
val metaProps = new MetaProperties.Builder().
setVersion(MetaPropertiesVersion.V0).
setClusterId("clusterId").
@@ -3384,7 +3384,7 @@ class ReplicaManagerTest {
new File(new File(dir), MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false)
})
}
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)), new LogConfig(logProps), log = if (shouldMockLog) Some(mockLog) else None, remoteStorageSystemEnable = enableRemoteStorage)
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)), new LogConfig(logProps), log = if (shouldMockLog) Some(mockLog) else None, remoteStorageSystemEnable = enableRemoteStorage)
val logConfig = new LogConfig(logProps)
when(mockLog.config).thenReturn(logConfig)
when(mockLog.remoteLogEnabled()).thenReturn(enableRemoteStorage)
@@ -3714,8 +3714,8 @@ class ReplicaManagerTest {
val config0 = KafkaConfig.fromProps(props0)
val config1 = KafkaConfig.fromProps(props1)
- val mockLogMgr0 = TestUtils.createLogManager(config0.logDirs.map(new File(_)))
- val mockLogMgr1 = TestUtils.createLogManager(config1.logDirs.map(new File(_)))
+ val mockLogMgr0 = TestUtils.createLogManager(config0.logDirs.asScala.map(new File(_)))
+ val mockLogMgr1 = TestUtils.createLogManager(config1.logDirs.asScala.map(new File(_)))
val metadataCache0: MetadataCache = mock(classOf[MetadataCache])
val metadataCache1: MetadataCache = mock(classOf[MetadataCache])
@@ -4592,7 +4592,7 @@ class ReplicaManagerTest {
def createReplicaManager(): ReplicaManager = {
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect)
val config = KafkaConfig.fromProps(props)
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
new ReplicaManager(
metrics = metrics,
config = config,
@@ -6499,7 +6499,7 @@ class ReplicaManagerTest {
@Test
def testCheckpointHwOnShutdown(): Unit = {
- val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)))
+ val mockLogMgr = TestUtils.createLogManager(config.logDirs.asScala.map(new File(_)))
val spyRm = spy(new ReplicaManager(
metrics = metrics,
config = config,
@@ -6656,7 +6656,7 @@ class ReplicaManagerTest {
// Given
val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect, logDirCount = 2)
val config = KafkaConfig.fromProps(props)
- val logDirFiles = config.logDirs.map(new File(_))
+ val logDirFiles = config.logDirs.asScala.map(new File(_))
val logDirFailureChannel = new LogDirFailureChannel(config.logDirs.size)
val logManager = TestUtils.createLogManager(logDirFiles, defaultConfig = new LogConfig(new Properties()), time = time)
val mockZkClient = mock(classOf[KafkaZkClient])
diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
index 6da56191b4df6..a36b6331acf89 100755
--- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala
@@ -158,7 +158,7 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness {
val startupException = assertThrows(classOf[RuntimeException], () => serverB2.startup())
assertTrue(startupException.getMessage.startsWith("Stored node id 1 doesn't match previous node id 2"),
"Unexpected exception message " + startupException.getMessage)
- serverB2.config.logDirs.foreach(logDir => Utils.delete(new File(logDir)))
+ serverB2.config.logDirs.forEach(logDir => Utils.delete(new File(logDir)))
propsB.setProperty(ServerConfigs.BROKER_ID_CONFIG, "3")
val serverB3 = new KafkaServer(KafkaConfig.fromProps(propsB),
threadNamePrefix = Option(this.getClass.getName))
@@ -174,8 +174,8 @@ class ServerGenerateBrokerIdTest extends QuorumTestHarness {
TestUtils.assertNoNonDaemonThreads(this.getClass.getName)
}
- def verifyBrokerMetadata(logDirs: Seq[String], brokerId: Int): Boolean = {
- for (logDir <- logDirs) {
+ def verifyBrokerMetadata(logDirs: java.util.List[String], brokerId: Int): Boolean = {
+ logDirs.forEach { logDir =>
val properties = PropertiesUtils.readPropertiesFile(
new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
val metaProps = new MetaProperties.Builder(properties).build()
diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
index a9851f45e0ead..b937bdbf8afbf 100755
--- a/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerGenerateClusterIdTest.scala
@@ -202,10 +202,8 @@ class ServerGenerateClusterIdTest extends QuorumTestHarness {
TestUtils.assertNoNonDaemonThreads(this.getClass.getName)
}
- def forgeBrokerMetadata(logDirs: Seq[String], brokerId: Int, clusterId: String): Unit = {
- for (logDir <- logDirs) {
- forgeBrokerMetadata(logDir, brokerId, clusterId)
- }
+ def forgeBrokerMetadata(logDirs: java.util.List[String], brokerId: Int, clusterId: String): Unit = {
+ logDirs.forEach(logDir => forgeBrokerMetadata(logDir, brokerId, clusterId))
}
def forgeBrokerMetadata(logDir: String, brokerId: Int, clusterId: String): Unit = {
@@ -218,8 +216,8 @@ class ServerGenerateClusterIdTest extends QuorumTestHarness {
new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath, false)
}
- def verifyBrokerMetadata(logDirs: Seq[String], clusterId: String): Boolean = {
- for (logDir <- logDirs) {
+ def verifyBrokerMetadata(logDirs: java.util.List[String], clusterId: String): Boolean = {
+ logDirs.forEach { logDir =>
val properties = PropertiesUtils.readPropertiesFile(
new File(logDir, MetaPropertiesEnsemble.META_PROPERTIES_NAME).getAbsolutePath)
val metaProps = new MetaProperties.Builder(properties).build()
diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
index a141d966268a4..b380a207771fb 100644
--- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala
@@ -113,7 +113,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
// do a clean shutdown and check that offset checkpoint file exists
shutdownBroker()
- for (logDir <- config.logDirs) {
+ for (logDir <- config.logDirs.asScala) {
val OffsetCheckpointFile = new File(logDir, LogManager.RecoveryPointCheckpointFile)
assertTrue(OffsetCheckpointFile.exists)
assertTrue(OffsetCheckpointFile.length() > 0)
@@ -163,7 +163,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(quorum: String): Unit = {
createTopic(topic)
shutdownBroker()
- config.logDirs.foreach { dirName =>
+ config.logDirs.forEach { dirName =>
val partitionDir = new File(dirName, s"$topic-0")
partitionDir.listFiles.foreach(f => TestUtils.appendNonsenseToFile(f, TestUtils.random.nextInt(1024) + 1))
}
diff --git a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
index 8ee91c592cc86..e59547732a53a 100644
--- a/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
+++ b/core/src/test/scala/unit/kafka/server/epoch/EpochDrivenReplicationProtocolAcceptanceTest.scala
@@ -152,7 +152,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
broker100.shutdown()
//Delete the clean shutdown file to simulate crash
- new File(broker100.config.logDirs.head, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).delete()
+ new File(broker100.config.logDirs.asScala.head, CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).delete()
//Delete 5 messages from the leader's log on 100
deleteMessagesFromLogFile(5 * msg.length, broker100, 0)
@@ -199,7 +199,7 @@ class EpochDrivenReplicationProtocolAcceptanceTest extends QuorumTestHarness wit
brokers.foreach { b => b.shutdown() }
//Delete the clean shutdown file to simulate crash
- new File(brokers(0).config.logDirs(0), CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).delete()
+ new File(brokers(0).config.logDirs.get(0), CleanShutdownFileHandler.CLEAN_SHUTDOWN_FILE_NAME).delete()
//Delete half the messages from the log file
deleteMessagesFromLogFile(getLogFile(brokers(0), 0).length() / 2, brokers(0), 0)
diff --git a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
index 8365fbd10ebe1..320324d03e365 100644
--- a/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/StorageToolTest.scala
@@ -671,7 +671,7 @@ Found problem:
"-t", "XcZZOzUqS4yHOjhMQB6JLQ",
"--release-version", MetadataVersion.LATEST_PRODUCTION.toString)
val exitCode = StorageTool.runFormatCommand(StorageTool.parseArguments(args), config)
- Mockito.verify(config, Mockito.times(1)).validateWithMetadataVersion(MetadataVersion.LATEST_PRODUCTION)
+ Mockito.verify(config, Mockito.times(1)).validator().validateWithMetadataVersion(MetadataVersion.LATEST_PRODUCTION)
assertEquals(0, exitCode)
}
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index dcd0de0e8be98..eee60dc29ba03 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -87,6 +87,7 @@ import java.util.{Arrays, Collections, Optional, Properties}
import scala.annotation.nowarn
import scala.collection.mutable.ArrayBuffer
import scala.collection.{Map, Seq, immutable, mutable}
+import scala.compat.java8.OptionConverters.RichOptionalGeneric
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.{Await, ExecutionContext, Future}
import scala.jdk.CollectionConverters._
@@ -237,7 +238,7 @@ object TestUtils extends Logging {
listenerName: ListenerName
): String = {
brokers.map { s =>
- val listener = s.config.effectiveAdvertisedListeners.find(_.listenerName == listenerName).getOrElse(
+ val listener = s.config.effectiveAdvertisedListeners.asScala.find(_.listenerName.asScala.contains(listenerName.value())).getOrElse(
sys.error(s"Could not find listener with name ${listenerName.value}"))
formatAddress(listener.host, s.boundPort(listenerName))
}.mkString(",")
@@ -1353,14 +1354,14 @@ object TestUtils extends Logging {
checkpoints.forall(checkpointsPerLogDir => !checkpointsPerLogDir.contains(tp))
}), "Cleaner offset for deleted partition should have been removed")
waitUntilTrue(() => brokers.forall(broker =>
- broker.config.logDirs.forall { logDir =>
+ broker.config.logDirs.stream().allMatch { logDir =>
topicPartitions.forall { tp =>
!new File(logDir, tp.topic + "-" + tp.partition).exists()
}
}
), "Failed to soft-delete the data to a delete directory")
waitUntilTrue(() => brokers.forall(broker =>
- broker.config.logDirs.forall { logDir =>
+ broker.config.logDirs.stream().allMatch { logDir =>
topicPartitions.forall { tp =>
!Arrays.asList(new File(logDir).list()).asScala.exists { partitionDirectoryName =>
partitionDirectoryName.startsWith(tp.topic + "-" + tp.partition) &&
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
index dbc9731ff07bd..fb65f4544dd4d 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/CheckpointBench.java
@@ -108,7 +108,7 @@ public void setup() {
this.time = new MockTime();
this.failureChannel = new LogDirFailureChannel(brokerProperties.logDirs().size());
final List files =
- JavaConverters.seqAsJavaList(brokerProperties.logDirs()).stream().map(File::new).collect(Collectors.toList());
+ brokerProperties.logDirs().stream().map(File::new).collect(Collectors.toList());
this.logManager = TestUtils.createLogManager(JavaConverters.asScalaBuffer(files),
new LogConfig(new Properties()), new MockConfigRepository(), new CleanerConfig(1, 4 * 1024 * 1024L, 0.9d,
1024 * 1024, 32 * 1024 * 1024, Double.MAX_VALUE, 15 * 1000, true), time, MetadataVersion.latestTesting(), 4, false, Option.empty(), false, ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT);
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
index f1af675c62857..9fe0c7d14a68c 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/server/PartitionCreationBench.java
@@ -118,7 +118,7 @@ public void setup() {
this.failureChannel = new LogDirFailureChannel(brokerProperties.logDirs().size());
final BrokerTopicStats brokerTopicStats = new BrokerTopicStats(false);
final List files =
- JavaConverters.seqAsJavaList(brokerProperties.logDirs()).stream().map(File::new).collect(Collectors.toList());
+ brokerProperties.logDirs().stream().map(File::new).collect(Collectors.toList());
CleanerConfig cleanerConfig = new CleanerConfig(1,
4 * 1024 * 1024L, 0.9d,
1024 * 1024, 32 * 1024 * 1024,
diff --git a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java
index 245562e0fc382..bf8b6d0564275 100644
--- a/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java
+++ b/server/src/main/java/org/apache/kafka/server/config/AbstractKafkaConfig.java
@@ -16,33 +16,60 @@
*/
package org.apache.kafka.server.config;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.TopicConfig;
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
+import org.apache.kafka.common.config.types.Password;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy;
+import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor;
import org.apache.kafka.coordinator.transaction.TransactionLogConfigs;
import org.apache.kafka.coordinator.transaction.TransactionStateManagerConfigs;
import org.apache.kafka.network.SocketServerConfigs;
import org.apache.kafka.raft.QuorumConfig;
import org.apache.kafka.security.PasswordEncoderConfigs;
+import org.apache.kafka.security.authorizer.AuthorizerUtils;
+import org.apache.kafka.server.ProcessRole;
+import org.apache.kafka.server.authorizer.Authorizer;
+import org.apache.kafka.server.common.MetadataVersion;
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig;
import org.apache.kafka.server.metrics.MetricConfigs;
+import org.apache.kafka.server.util.Csv;
+import org.apache.kafka.server.utils.EndpointUtils;
import org.apache.kafka.storage.internals.log.CleanerConfig;
import org.apache.kafka.storage.internals.log.LogConfig;
-
+import java.util.AbstractMap;
import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
/**
- * During moving {@link kafka.server.KafkaConfig} out of core AbstractKafkaConfig will be the future KafkaConfig
+ * During moving kafka.server.KafkaConfig out of core AbstractKafkaConfig will be the future KafkaConfig
* so any new getters, or updates to `CONFIG_DEF` will be defined here.
* Any code depends on kafka.server.KafkaConfig will keep for using kafka.server.KafkaConfig for the time being until we move it out of core
* For more details check KAFKA-15853
*/
public abstract class AbstractKafkaConfig extends AbstractConfig {
- @SuppressWarnings("deprecation")
- public static final ConfigDef CONFIG_DEF = Utils.mergeConfigs(Arrays.asList(
+ public static final ConfigDef CONFIG_DEF = Utils.mergeConfigs(Arrays.asList(
RemoteLogManagerConfig.configDef(),
ZkConfigs.CONFIG_DEF,
ServerConfigs.CONFIG_DEF,
@@ -64,8 +91,1368 @@ public abstract class AbstractKafkaConfig extends AbstractConfig {
BrokerSecurityConfigs.CONFIG_DEF,
DelegationTokenManagerConfigs.CONFIG_DEF,
PasswordEncoderConfigs.CONFIG_DEF
- ));
- public AbstractKafkaConfig(ConfigDef definition, Map, ?> originals, Map configProviderProps, boolean doLog) {
- super(definition, originals, configProviderProps, doLog);
+ ));
+ private static final Logger log = LoggerFactory.getLogger(AbstractKafkaConfig.class);
+ protected final KafkaConfigValidator configValidator;
+ private final RemoteLogManagerConfig remoteLogManagerConfig;
+
+ private Integer brokerId;
+
+ // Cache the current config to avoid acquiring read lock to access from dynamicConfig
+ protected volatile AbstractKafkaConfig currentConfig = this;
+
+ public AbstractKafkaConfig(ConfigDef definition, Map, ?> original, Map configProviderProps, boolean doLog) {
+ super(definition, original, configProviderProps, doLog);
+ this.remoteLogManagerConfig = new RemoteLogManagerConfig(original);
+ this.configValidator = new KafkaConfigValidator(this.currentConfig, log);
+ }
+
+ public KafkaConfigValidator validator() {
+ return configValidator;
+ }
+
+ public Map valuesFromThisConfigWithPrefixOverride(String prefix) {
+ return super.valuesWithPrefixOverride(prefix);
+ }
+
+ public Map valuesFromThisConfig() {
+ return super.values();
+ }
+
+ // During dynamic update, we use the values from this config, these are only used in DynamicBrokerConfig
+ public Map originalsFromThisConfig() {
+ return super.originals();
+ }
+
+ @Override
+ public Map originals() {
+ return (this == currentConfig) ? super.originals() : currentConfig.originals();
+ }
+
+ @Override
+ public Map values() {
+ return (this == currentConfig) ? super.values() : currentConfig.values();
+ }
+
+ @Override
+ public Map nonInternalValues() {
+ return (this == currentConfig) ? super.nonInternalValues() : currentConfig.nonInternalValues();
+ }
+
+ @Override
+ public Map originalsStrings() {
+ return (this == currentConfig) ? super.originalsStrings() : currentConfig.originalsStrings();
+ }
+
+ @Override
+ public Map originalsWithPrefix(String prefix) {
+ return (this == currentConfig) ? super.originalsWithPrefix(prefix) : currentConfig.originalsWithPrefix(prefix);
+ }
+
+ @Override
+ public Map valuesWithPrefixOverride(String prefix) {
+ return (this == currentConfig) ? super.valuesWithPrefixOverride(prefix) : currentConfig.valuesWithPrefixOverride(prefix);
+ }
+
+ @Override
+ public Object get(String key) {
+ return (this == currentConfig) ? super.get(key) : currentConfig.get(key);
+ }
+
+
+ public final RemoteLogManagerConfig remoteLogManagerConfig() {
+ return remoteLogManagerConfig;
+ }
+
+ /**
+ * ******** Socket Server Configuration
+ ***********/
+ public final int socketSendBufferBytes() {
+ return getInt(SocketServerConfigs.SOCKET_SEND_BUFFER_BYTES_CONFIG);
+ }
+
+ public final int socketReceiveBufferBytes() {
+ return getInt(SocketServerConfigs.SOCKET_RECEIVE_BUFFER_BYTES_CONFIG);
+ }
+
+ public final int socketRequestMaxBytes() {
+ return getInt(SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG);
+ }
+
+ public final int socketListenBacklogSize() {
+ return getInt(SocketServerConfigs.SOCKET_LISTEN_BACKLOG_SIZE_CONFIG);
+ }
+
+ public final int maxConnectionsPerIp() {
+ return getInt(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG);
+ }
+
+ public final Map maxConnectionsPerIpOverrides() {
+ return getMap(
+ SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG,
+ getString(SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG)
+ ).entrySet().stream()
+ .collect(Collectors.toMap(Map.Entry::getKey, entry -> Integer.parseInt(entry.getValue())));
+ }
+
+ public final int maxConnections() {
+ return getInt(SocketServerConfigs.MAX_CONNECTIONS_CONFIG);
+ }
+
+ public final int maxConnectionCreationRate() {
+ return getInt(SocketServerConfigs.MAX_CONNECTION_CREATION_RATE_CONFIG);
+ }
+
+ public final long connectionsMaxIdleMs() {
+ return getLong(SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG);
+ }
+
+ public final int failedAuthenticationDelayMs() {
+ return getInt(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG);
+ }
+
+ /***************** rack configuration **************/
+ public final Optional rack() {
+ return Optional.ofNullable(getString(ServerConfigs.BROKER_RACK_CONFIG));
+ }
+
+ public final Optional replicaSelectorClassName() {
+ return Optional.ofNullable(getString(ReplicationConfigs.REPLICA_SELECTOR_CLASS_CONFIG));
+ }
+
+ /**
+ * ******** Log Configuration
+ ***********/
+ public final Boolean autoCreateTopicsEnable() {
+ return getBoolean(ServerLogConfigs.AUTO_CREATE_TOPICS_ENABLE_CONFIG);
+ }
+
+ public final int numPartitions() {
+ return getInt(ServerLogConfigs.NUM_PARTITIONS_CONFIG);
+ }
+
+ public final List logDirs() {
+ return Csv.parseCsvList(Optional.ofNullable(getString(ServerLogConfigs.LOG_DIRS_CONFIG)).orElse(getString(ServerLogConfigs.LOG_DIR_CONFIG)));
+ }
+
+ public final int logSegmentBytes() {
+ return getInt(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG);
+ }
+
+ public final long logFlushIntervalMessages() {
+ return getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MESSAGES_CONFIG);
+ }
+
+ public final int logCleanerThreads() {
+ return getInt(CleanerConfig.LOG_CLEANER_THREADS_PROP);
+ }
+
+ public final int numRecoveryThreadsPerDataDir() {
+ return getInt(ServerLogConfigs.NUM_RECOVERY_THREADS_PER_DATA_DIR_CONFIG);
+ }
+
+ public final long logFlushSchedulerIntervalMs() {
+ return getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG);
+ }
+
+ public final long logFlushOffsetCheckpointIntervalMs() {
+ return (long) getInt(ServerLogConfigs.LOG_FLUSH_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int logFlushStartOffsetCheckpointIntervalMs() {
+ return getInt(ServerLogConfigs.LOG_FLUSH_START_OFFSET_CHECKPOINT_INTERVAL_MS_CONFIG);
+ }
+
+ public final long logCleanupIntervalMs() {
+ return getLong(ServerLogConfigs.LOG_CLEANUP_INTERVAL_MS_CONFIG);
+ }
+
+ public final List logCleanupPolicy() {
+ return getList(ServerLogConfigs.LOG_CLEANUP_POLICY_CONFIG);
+ }
+
+ public final int offsetsRetentionMinutes() {
+ return getInt(GroupCoordinatorConfig.OFFSETS_RETENTION_MINUTES_CONFIG);
+ }
+
+ public final long offsetsRetentionCheckIntervalMs() {
+ return getLong(GroupCoordinatorConfig.OFFSETS_RETENTION_CHECK_INTERVAL_MS_CONFIG);
+ }
+
+ public final long logRetentionBytes() {
+ return getLong(ServerLogConfigs.LOG_RETENTION_BYTES_CONFIG);
+ }
+
+ public final long logCleanerDedupeBufferSize() {
+ return getLong(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_SIZE_PROP);
+ }
+
+ public final double logCleanerDedupeBufferLoadFactor() {
+ return getDouble(CleanerConfig.LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP);
+ }
+
+ public final int logCleanerIoBufferSize() {
+ return getInt(CleanerConfig.LOG_CLEANER_IO_BUFFER_SIZE_PROP);
+ }
+
+ public final double logCleanerIoMaxBytesPerSecond() {
+ return getDouble(CleanerConfig.LOG_CLEANER_IO_MAX_BYTES_PER_SECOND_PROP);
+ }
+
+ public final long logCleanerDeleteRetentionMs() {
+ return getLong(CleanerConfig.LOG_CLEANER_DELETE_RETENTION_MS_PROP);
+ }
+
+ public final long logCleanerMinCompactionLagMs() {
+ return getLong(CleanerConfig.LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP);
+ }
+
+ public final long logCleanerMaxCompactionLagMs() {
+ return getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP);
+ }
+
+ public final long logCleanerBackoffMs() {
+ return getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP);
+ }
+
+ public final double logCleanerMinCleanRatio() {
+ return getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP);
+ }
+
+ public final boolean logCleanerEnable() {
+ return getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP);
+ }
+
+ public final int logIndexSizeMaxBytes() {
+ return getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG);
+ }
+
+ public final int logIndexIntervalBytes() {
+ return getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG);
+ }
+
+ public final long logDeleteDelayMs() {
+ return getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG);
+ }
+
+ public final Long logRollTimeMillis() {
+ return Optional.ofNullable(getLong(ServerLogConfigs.LOG_ROLL_TIME_MILLIS_CONFIG)).orElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_HOURS_CONFIG));
+ }
+
+ public final Long logRollTimeJitterMillis() {
+ return Optional.ofNullable(getLong(ServerLogConfigs.LOG_ROLL_TIME_JITTER_MILLIS_CONFIG)).orElse(60 * 60 * 1000L * getInt(ServerLogConfigs.LOG_ROLL_TIME_JITTER_HOURS_CONFIG));
+ }
+
+ public final Long logFlushIntervalMs() {
+ return Optional.ofNullable(getLong(ServerLogConfigs.LOG_FLUSH_INTERVAL_MS_CONFIG)).orElse(getLong(ServerLogConfigs.LOG_FLUSH_SCHEDULER_INTERVAL_MS_CONFIG));
+ }
+
+ public final int minInSyncReplicas() {
+ return getInt(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG);
+ }
+
+ public final boolean logPreAllocateEnable() {
+ return getBoolean(ServerLogConfigs.LOG_PRE_ALLOCATE_CONFIG);
+ }
+
+ public final Long logInitialTaskDelayMs() {
+ return Optional.ofNullable(getLong(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_CONFIG)).orElse(ServerLogConfigs.LOG_INITIAL_TASK_DELAY_MS_DEFAULT);
+ }
+
+ // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version
+ // (e.g. if `0.10.0` is passed, `0.10.0-IV0` may be picked)
+ @Deprecated
+ public final String logMessageFormatVersionString() {
+ return getString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG);
+ }
+
+ /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details
+ deprecated since 3.0
+ */
+ @Deprecated
+ public final MetadataVersion logMessageFormatVersion() {
+ if (LogConfig.shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion()))
+ return MetadataVersion.fromVersionString(ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_DEFAULT);
+ else return MetadataVersion.fromVersionString(logMessageFormatVersionString());
+ }
+
+ public final TimestampType logMessageTimestampType() {
+ return TimestampType.forName(getString(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_TYPE_CONFIG));
+ }
+
+ /* See `TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG` for details
+ deprecated since 3.6
+ */
+ @Deprecated
+ public final long logMessageTimestampDifferenceMaxMs() {
+ return getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
+ }
+
+ // In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
+ // we are using its value if logMessageTimestampBeforeMaxMs default value hasn't changed.
+ // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
+ @Deprecated
+ public final long logMessageTimestampBeforeMaxMs() {
+ long messageTimestampBeforeMaxMs = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG);
+ if (messageTimestampBeforeMaxMs != ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DEFAULT) {
+ return messageTimestampBeforeMaxMs;
+ } else {
+ return logMessageTimestampDifferenceMaxMs();
+ }
+ }
+
+ // In the transition period before logMessageTimestampDifferenceMaxMs is removed, to maintain backward compatibility,
+ // we are using its value if logMessageTimestampAfterMaxMs default value hasn't changed.
+ // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
+ @Deprecated
+ public final long logMessageTimestampAfterMaxMs() {
+ long messageTimestampAfterMaxMs = getLong(ServerLogConfigs.LOG_MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG);
+ if (messageTimestampAfterMaxMs != Long.MAX_VALUE) {
+ return messageTimestampAfterMaxMs;
+ } else {
+ return logMessageTimestampDifferenceMaxMs();
+ }
+ }
+
+ public final boolean logMessageDownConversionEnable() {
+ return getBoolean(ServerLogConfigs.LOG_MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
+ }
+
+ public final long logDirFailureTimeoutMs() {
+ return getLong(ServerLogConfigs.LOG_DIR_FAILURE_TIMEOUT_MS_CONFIG);
+ }
+
+ /**
+ * ******** Replication configuration
+ ***********/
+ public final int controllerSocketTimeoutMs() {
+ return getInt(ReplicationConfigs.CONTROLLER_SOCKET_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int defaultReplicationFactor() {
+ return getInt(ReplicationConfigs.DEFAULT_REPLICATION_FACTOR_CONFIG);
+ }
+
+ public final long replicaLagTimeMaxMs() {
+ return getLong(ReplicationConfigs.REPLICA_LAG_TIME_MAX_MS_CONFIG);
+ }
+
+ public final int replicaSocketTimeoutMs() {
+ return getInt(ReplicationConfigs.REPLICA_SOCKET_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int replicaSocketReceiveBufferBytes() {
+ return getInt(ReplicationConfigs.REPLICA_SOCKET_RECEIVE_BUFFER_BYTES_CONFIG);
+ }
+
+ public final int replicaFetchMaxBytes() {
+ return getInt(ReplicationConfigs.REPLICA_FETCH_MAX_BYTES_CONFIG);
+ }
+
+ public final int replicaFetchWaitMaxMs() {
+ return getInt(ReplicationConfigs.REPLICA_FETCH_WAIT_MAX_MS_CONFIG);
+ }
+
+ public final int replicaFetchMinBytes() {
+ return getInt(ReplicationConfigs.REPLICA_FETCH_MIN_BYTES_CONFIG);
+ }
+
+ public final int replicaFetchResponseMaxBytes() {
+ return getInt(ReplicationConfigs.REPLICA_FETCH_RESPONSE_MAX_BYTES_CONFIG);
+ }
+
+ public final int replicaFetchBackoffMs() {
+ return getInt(ReplicationConfigs.REPLICA_FETCH_BACKOFF_MS_CONFIG);
+ }
+
+ public final int numReplicaFetchers() {
+ return getInt(ReplicationConfigs.NUM_REPLICA_FETCHERS_CONFIG);
+ }
+
+ public final long replicaHighWatermarkCheckpointIntervalMs() {
+ return getLong(ReplicationConfigs.REPLICA_HIGH_WATERMARK_CHECKPOINT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int fetchPurgatoryPurgeIntervalRequests() {
+ return getInt(ReplicationConfigs.FETCH_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG);
+ }
+
+ public final int producerPurgatoryPurgeIntervalRequests() {
+ return getInt(ReplicationConfigs.PRODUCER_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG);
+ }
+
+ public final int deleteRecordsPurgatoryPurgeIntervalRequests() {
+ return getInt(ReplicationConfigs.DELETE_RECORDS_PURGATORY_PURGE_INTERVAL_REQUESTS_CONFIG);
+ }
+
+ public final boolean autoLeaderRebalanceEnable() {
+ return getBoolean(ReplicationConfigs.AUTO_LEADER_REBALANCE_ENABLE_CONFIG);
+ }
+
+ public final int leaderImbalancePerBrokerPercentage() {
+ return getInt(ReplicationConfigs.LEADER_IMBALANCE_PER_BROKER_PERCENTAGE_CONFIG);
+ }
+
+ public final long leaderImbalanceCheckIntervalSeconds() {
+ return getLong(ReplicationConfigs.LEADER_IMBALANCE_CHECK_INTERVAL_SECONDS_CONFIG);
+ }
+
+ public final boolean uncleanLeaderElectionEnable() {
+ return getBoolean(ReplicationConfigs.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG);
+ }
+
+ // We keep the user-provided String as `MetadataVersion.fromVersionString` can choose a slightly different version (eg if `0.10.0`
+ // is passed, `0.10.0-IV0` may be picked)
+ public final String interBrokerProtocolVersionString() {
+ return getString(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG);
+ }
+
+ public final MetadataVersion interBrokerProtocolVersion() {
+ if (processRoles().isEmpty()) {
+ return MetadataVersion.fromVersionString(interBrokerProtocolVersionString());
+ } else {
+ if (originals().containsKey(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG)) {
+ // A user-supplied IBP was given
+ MetadataVersion configuredVersion = MetadataVersion.fromVersionString(interBrokerProtocolVersionString());
+ if (!configuredVersion.isKRaftSupported()) {
+ throw new ConfigException(String.format("A non-KRaft version %s given for %s. The minimum version is %s",
+ interBrokerProtocolVersionString(), ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG, MetadataVersion.MINIMUM_KRAFT_VERSION));
+ } else {
+ log.warn(ReplicationConfigs.INTER_BROKER_PROTOCOL_VERSION_CONFIG + " is deprecated in KRaft mode as of 3.3 and will only " +
+ "be read when first upgrading from a KRaft prior to 3.3. See kafka-storage.sh help for details on setting " +
+ "the metadata.version for a new KRaft cluster.");
+ }
+ }
+ // In KRaft mode, we pin this value to the minimum KRaft-supported version. This prevents inadvertent usage of
+ // the static IBP config in broker components running in KRaft mode
+ return MetadataVersion.MINIMUM_KRAFT_VERSION;
+ }
+ }
+
+ /**
+ * ******** Controlled shutdown configuration
+ ***********/
+ public final int controlledShutdownMaxRetries() {
+ return getInt(ServerConfigs.CONTROLLED_SHUTDOWN_MAX_RETRIES_CONFIG);
+ }
+
+ public final long controlledShutdownRetryBackoffMs() {
+ return getLong(ServerConfigs.CONTROLLED_SHUTDOWN_RETRY_BACKOFF_MS_CONFIG);
+ }
+
+ public final boolean controlledShutdownEnable() {
+ return getBoolean(ServerConfigs.CONTROLLED_SHUTDOWN_ENABLE_CONFIG);
+ }
+
+ /**
+ * ******** Feature configuration
+ ***********/
+ public final boolean isFeatureVersioningSupported() {
+ return interBrokerProtocolVersion().isFeatureVersioningSupported();
+ }
+
+ /**
+ * ******** Group coordinator configuration
+ ***********/
+ public final int groupMinSessionTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int groupMaxSessionTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int groupInitialRebalanceDelay() {
+ return getInt(GroupCoordinatorConfig.GROUP_INITIAL_REBALANCE_DELAY_MS_CONFIG);
+ }
+
+ public final int groupMaxSize() {
+ return getInt(GroupCoordinatorConfig.GROUP_MAX_SIZE_CONFIG);
+ }
+
+ /**
+ * New group coordinator configs
+ */
+ public final Set groupCoordinatorRebalanceProtocols() {
+ Set protocols = getList(GroupCoordinatorConfig.GROUP_COORDINATOR_REBALANCE_PROTOCOLS_CONFIG).stream()
+ .map(String::toUpperCase)
+ .map(Group.GroupType::valueOf)
+ .collect(Collectors.toSet());
+ if (!protocols.contains(Group.GroupType.CLASSIC)) {
+ throw new ConfigException(String.format("Disabling the '%s' protocol is not supported.", Group.GroupType.CLASSIC));
+ }
+ if (protocols.contains(Group.GroupType.CONSUMER)) {
+ log.warn(String.format("The new '%s' rebalance protocol is enabled along with the new group coordinator. " +
+ "This is part of the early access of KIP-848 and MUST NOT be used in production.", Group.GroupType.CONSUMER));
+ }
+ return protocols;
+ }
+
+ // The new group coordinator is enabled in two cases: 1) The internal configuration to enable
+ // it is explicitly set; or 2) the consumer rebalance protocol is enabled.
+ public boolean isNewGroupCoordinatorEnabled() {
+ return getBoolean(GroupCoordinatorConfig.NEW_GROUP_COORDINATOR_ENABLE_CONFIG) ||
+ groupCoordinatorRebalanceProtocols().contains(Group.GroupType.CONSUMER);
+ }
+
+ public final int groupCoordinatorNumThreads() {
+ return getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_NUM_THREADS_CONFIG);
+ }
+
+ public final int groupCoordinatorAppendLingerMs() {
+ return getInt(GroupCoordinatorConfig.GROUP_COORDINATOR_APPEND_LINGER_MS_CONFIG);
+ }
+
+ /**
+ * Consumer group configs
+ */
+ public final int consumerGroupSessionTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int consumerGroupMinSessionTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int consumerGroupMaxSessionTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int consumerGroupHeartbeatIntervalMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int consumerGroupMinHeartbeatIntervalMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int consumerGroupMaxHeartbeatIntervalMs() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int consumerGroupMaxSize() {
+ return getInt(GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SIZE_CONFIG);
+ }
+
+ public final List consumerGroupAssignors() {
+ return getConfiguredInstances(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, ConsumerGroupPartitionAssignor.class);
+ }
+
+ public final ConsumerGroupMigrationPolicy consumerGroupMigrationPolicy() {
+ return ConsumerGroupMigrationPolicy.parse(getString(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG));
+ }
+
+ /**
+ * Share group configuration
+ **/
+ public final boolean isShareGroupEnabled() {
+ return getBoolean(ShareGroupConfigs.SHARE_GROUP_ENABLE_CONFIG);
+ }
+
+ public final int shareGroupPartitionMaxRecordLocks() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_PARTITION_MAX_RECORD_LOCKS_CONFIG);
+ }
+
+ public final int shareGroupDeliveryCountLimit() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_DELIVERY_COUNT_LIMIT_CONFIG);
+ }
+
+ public final short shareGroupMaxGroups() {
+ return getShort(ShareGroupConfigs.SHARE_GROUP_MAX_GROUPS_CONFIG);
+ }
+
+ public final short shareGroupMaxSize() {
+ return getShort(ShareGroupConfigs.SHARE_GROUP_MAX_SIZE_CONFIG);
+ }
+
+ public final int shareGroupSessionTimeoutMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int shareGroupMinSessionTimeoutMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int shareGroupMaxSessionTimeoutMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int shareGroupHeartbeatIntervalMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int shareGroupMinHeartbeatIntervalMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int shareGroupMaxHeartbeatIntervalMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int shareGroupRecordLockDurationMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG);
+ }
+
+ public final int shareGroupMaxRecordLockDurationMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG);
+ }
+
+ public final int shareGroupMinRecordLockDurationMs() {
+ return getInt(ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG);
+ }
+
+
+ /**
+ * ******** Offset management configuration
+ ***********/
+ public final int offsetMetadataMaxSize() {
+ return getInt(GroupCoordinatorConfig.OFFSET_METADATA_MAX_SIZE_CONFIG);
+ }
+
+ public final int offsetsLoadBufferSize() {
+ return getInt(GroupCoordinatorConfig.OFFSETS_LOAD_BUFFER_SIZE_CONFIG);
+ }
+
+ public short offsetsTopicReplicationFactor() {
+ return getShort(GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG);
+ }
+
+ public final int offsetsTopicPartitions() {
+ return getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG);
+ }
+
+ public final int offsetCommitTimeoutMs() {
+ return getInt(GroupCoordinatorConfig.OFFSET_COMMIT_TIMEOUT_MS_CONFIG);
+ }
+
+ // Deprecated since 3.8
+ @Deprecated
+ public short offsetCommitRequiredAcks() {
+ return getShort(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG);
+ }
+
+ public final int offsetsTopicSegmentBytes() {
+ return getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_SEGMENT_BYTES_CONFIG);
+ }
+
+ public CompressionType offsetsTopicCompressionType() {
+ return Optional.ofNullable(getInt(GroupCoordinatorConfig.OFFSETS_TOPIC_COMPRESSION_CODEC_CONFIG))
+ .map(CompressionType::forId).orElse(null);
+ }
+
+ /**
+ * ******** Transaction management configuration
+ ***********/
+ public final int transactionalIdExpirationMs() {
+ return getInt(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG);
+ }
+
+ public final int transactionMaxTimeoutMs() {
+ return getInt(TransactionStateManagerConfigs.TRANSACTIONS_MAX_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int transactionTopicMinISR() {
+ return getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG);
+ }
+
+ public final int transactionsLoadBufferSize() {
+ return getInt(TransactionLogConfigs.TRANSACTIONS_LOAD_BUFFER_SIZE_CONFIG);
+ }
+
+ public short transactionTopicReplicationFactor() {
+ return getShort(TransactionLogConfigs.TRANSACTIONS_TOPIC_REPLICATION_FACTOR_CONFIG);
+ }
+
+ public final int transactionTopicPartitions() {
+ return getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_PARTITIONS_CONFIG);
+ }
+
+ public final int transactionTopicSegmentBytes() {
+ return getInt(TransactionLogConfigs.TRANSACTIONS_TOPIC_SEGMENT_BYTES_CONFIG);
+ }
+
+ public final int transactionAbortTimedOutTransactionCleanupIntervalMs() {
+ return getInt(TransactionStateManagerConfigs.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG);
+ }
+
+ public final int transactionRemoveExpiredTransactionalIdCleanupIntervalMs() {
+ return getInt(TransactionStateManagerConfigs.TRANSACTIONS_REMOVE_EXPIRED_TRANSACTIONAL_ID_CLEANUP_INTERVAL_MS_CONFIG);
+ }
+
+ public boolean transactionPartitionVerificationEnable() {
+ return getBoolean(TransactionLogConfigs.TRANSACTION_PARTITION_VERIFICATION_ENABLE_CONFIG);
+ }
+
+ public final int producerIdExpirationMs() {
+ return getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_MS_CONFIG);
+ }
+
+ public final int producerIdExpirationCheckIntervalMs() {
+ return getInt(TransactionLogConfigs.PRODUCER_ID_EXPIRATION_CHECK_INTERVAL_MS_CONFIG);
+ }
+
+ /**
+ * ******** Metric Configuration
+ **************/
+ public final int metricNumSamples() {
+ return getInt(MetricConfigs.METRIC_NUM_SAMPLES_CONFIG);
+ }
+
+ public final long metricSampleWindowMs() {
+ return getLong(MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG);
+ }
+
+ public final String metricRecordingLevel() {
+ return getString(MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG);
+ }
+
+ /**
+ * ******** Kafka Client Telemetry Metrics Configuration
+ ***********/
+ public final int clientTelemetryMaxBytes() {
+ return getInt(MetricConfigs.CLIENT_TELEMETRY_MAX_BYTES_CONFIG);
+ }
+
+ /**
+ * ******** SSL/SASL Configuration
+ **************/
+ // Security configs may be overridden for listeners, so it is not safe to use the base values
+ // Hence the base SSL/SASL configs are not fields of KafkaConfig, listener configs should be
+ // retrieved using KafkaConfig#valuesWithPrefixOverride
+ @SuppressWarnings("unchecked")
+ protected final Set saslEnabledMechanisms(ListenerName listenerName) {
+ return Optional.ofNullable(valuesWithPrefixOverride(listenerName.configPrefix()).get(BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG))
+ .map(value -> new HashSet<>((List) value))
+ .orElse(new HashSet<>());
+ }
+
+ public final ListenerName interBrokerListenerName() {
+ return getInterBrokerListenerNameAndSecurityProtocol().getKey();
+ }
+
+ public final SecurityProtocol interBrokerSecurityProtocol() {
+ return getInterBrokerListenerNameAndSecurityProtocol().getValue();
+ }
+
+ public final Optional controlPlaneListenerName() {
+ return getControlPlaneListenerNameAndSecurityProtocol().map(Map.Entry::getKey);
+ }
+
+ public final Optional controlPlaneSecurityProtocol() {
+ return getControlPlaneListenerNameAndSecurityProtocol().map(Map.Entry::getValue);
+ }
+
+ public final String saslMechanismInterBrokerProtocol() {
+ return getString(BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG);
+ }
+
+ public final boolean saslInterBrokerHandshakeRequestEnable() {
+ return interBrokerProtocolVersion().isSaslInterBrokerHandshakeRequestEnabled();
+ }
+
+ /**
+ * ******** DelegationToken Configuration
+ **************/
+ @SuppressWarnings("deprecation")
+ public final Password delegationTokenSecretKey() {
+ return Optional.ofNullable(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_CONFIG))
+ .orElse(getPassword(DelegationTokenManagerConfigs.DELEGATION_TOKEN_SECRET_KEY_ALIAS_CONFIG));
+ }
+
+ public final boolean tokenAuthEnabled() {
+ return delegationTokenSecretKey() != null && !delegationTokenSecretKey().value().isEmpty();
+ }
+
+ public final long delegationTokenMaxLifeMs() {
+ return getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_MAX_LIFETIME_CONFIG);
+ }
+
+ public final long delegationTokenExpiryTimeMs() {
+ return getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_TIME_MS_CONFIG);
+ }
+
+ public final long delegationTokenExpiryCheckIntervalMs() {
+ return getLong(DelegationTokenManagerConfigs.DELEGATION_TOKEN_EXPIRY_CHECK_INTERVAL_MS_CONFIG);
+ }
+
+ /**
+ * ******** Password encryption configuration for dynamic configs
+ *********/
+ public final Optional passwordEncoderSecret() {
+ return Optional.ofNullable(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_SECRET_CONFIG));
+ }
+
+ public final Optional passwordEncoderOldSecret() {
+ return Optional.ofNullable(getPassword(PasswordEncoderConfigs.PASSWORD_ENCODER_OLD_SECRET_CONFIG));
+ }
+
+ public final String passwordEncoderCipherAlgorithm() {
+ return getString(PasswordEncoderConfigs.PASSWORD_ENCODER_CIPHER_ALGORITHM_CONFIG);
+ }
+
+ public final String passwordEncoderKeyFactoryAlgorithm() {
+ return getString(PasswordEncoderConfigs.PASSWORD_ENCODER_KEYFACTORY_ALGORITHM_CONFIG);
+ }
+
+ public final int passwordEncoderKeyLength() {
+ return getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_KEY_LENGTH_CONFIG);
+ }
+
+ public final int passwordEncoderIterations() {
+ return getInt(PasswordEncoderConfigs.PASSWORD_ENCODER_ITERATIONS_CONFIG);
+ }
+
+ /** Quota Configuration **/
+ public final int numQuotaSamples() {
+ return getInt(QuotaConfigs.NUM_QUOTA_SAMPLES_CONFIG);
+ }
+
+ public final List quorumBootstrapServers() {
+ return getList(QuorumConfig.QUORUM_BOOTSTRAP_SERVERS_CONFIG);
+ }
+
+ public final int quotaWindowSizeSeconds() {
+ return getInt(QuotaConfigs.QUOTA_WINDOW_SIZE_SECONDS_CONFIG);
+ }
+
+ public final int numReplicationQuotaSamples() {
+ return getInt(QuotaConfigs.NUM_REPLICATION_QUOTA_SAMPLES_CONFIG);
+ }
+
+ public final int replicationQuotaWindowSizeSeconds() {
+ return getInt(QuotaConfigs.REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG);
+ }
+
+ public final int numAlterLogDirsReplicationQuotaSamples() {
+ return getInt(QuotaConfigs.NUM_ALTER_LOG_DIRS_REPLICATION_QUOTA_SAMPLES_CONFIG);
+ }
+
+ public final int alterLogDirsReplicationQuotaWindowSizeSeconds() {
+ return getInt(QuotaConfigs.ALTER_LOG_DIRS_REPLICATION_QUOTA_WINDOW_SIZE_SECONDS_CONFIG);
+ }
+
+ public final int numControllerQuotaSamples() {
+ return getInt(QuotaConfigs.NUM_CONTROLLER_QUOTA_SAMPLES_CONFIG);
+ }
+
+ public final int controllerQuotaWindowSizeSeconds() {
+ return getInt(QuotaConfigs.CONTROLLER_QUOTA_WINDOW_SIZE_SECONDS_CONFIG);
+ }
+
+ /**
+ * ******** Fetch Configuration
+ **************/
+ public final int maxIncrementalFetchSessionCacheSlots() {
+ return getInt(ServerConfigs.MAX_INCREMENTAL_FETCH_SESSION_CACHE_SLOTS_CONFIG);
+ }
+
+ public final int fetchMaxBytes() {
+ return getInt(ServerConfigs.FETCH_MAX_BYTES_CONFIG);
+ }
+
+ /**
+ * ******** Request Limit Configuration
+ ***********/
+ public final int maxRequestPartitionSizeLimit() {
+ return getInt(ServerConfigs.MAX_REQUEST_PARTITION_SIZE_LIMIT_CONFIG);
+ }
+
+ public final boolean deleteTopicEnable() {
+ return getBoolean(ServerConfigs.DELETE_TOPIC_ENABLE_CONFIG);
+ }
+
+ public final String compressionType() {
+ return getString(ServerConfigs.COMPRESSION_TYPE_CONFIG);
+ }
+
+ public final int gzipCompressionLevel() {
+ return getInt(ServerConfigs.COMPRESSION_GZIP_LEVEL_CONFIG);
+ }
+
+ public final int lz4CompressionLevel() {
+ return getInt(ServerConfigs.COMPRESSION_LZ4_LEVEL_CONFIG);
+ }
+
+ public final int zstdCompressionLevel() {
+ return getInt(ServerConfigs.COMPRESSION_ZSTD_LEVEL_CONFIG);
+ }
+
+ /**
+ * ******** Raft Quorum Configuration
+ *********/
+ public final List quorumVoters() {
+ return getList(QuorumConfig.QUORUM_VOTERS_CONFIG);
+ }
+
+ public final int quorumElectionTimeoutMs() {
+ return getInt(QuorumConfig.QUORUM_ELECTION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int quorumFetchTimeoutMs() {
+ return getInt(QuorumConfig.QUORUM_FETCH_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int quorumElectionBackoffMs() {
+ return getInt(QuorumConfig.QUORUM_ELECTION_BACKOFF_MAX_MS_CONFIG);
+ }
+
+ public final int quorumLingerMs() {
+ return getInt(QuorumConfig.QUORUM_LINGER_MS_CONFIG);
+ }
+
+ public final int quorumRequestTimeoutMs() {
+ return getInt(QuorumConfig.QUORUM_REQUEST_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int quorumRetryBackoffMs() {
+ return getInt(QuorumConfig.QUORUM_RETRY_BACKOFF_MS_CONFIG);
+ }
+
+ /**
+ * Internal Configurations
+ **/
+ public final boolean unstableApiVersionsEnabled() {
+ return getBoolean(ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG);
+ }
+
+ public final boolean unstableFeatureVersionsEnabled() {
+ return getBoolean(ServerConfigs.UNSTABLE_FEATURE_VERSIONS_ENABLE_CONFIG);
+ }
+
+ /**
+ * ******** General Configuration
+ ***********/
+ public Boolean brokerIdGenerationEnable() {
+ return getBoolean(ServerConfigs.BROKER_ID_GENERATION_ENABLE_CONFIG);
+ }
+
+ public final int maxReservedBrokerId() {
+ return getInt(ServerConfigs.RESERVED_BROKER_MAX_ID_CONFIG);
+ }
+
+ public final int brokerId() {
+ this.brokerId = getInt(ServerConfigs.BROKER_ID_CONFIG);
+ return this.brokerId;
+ }
+
+ public final void brokerId(int brokerId) {
+ this.brokerId = brokerId;
+ }
+
+ public final int nodeId() {
+ return getInt(KRaftConfigs.NODE_ID_CONFIG);
+ }
+
+ public final int initialRegistrationTimeoutMs() {
+ return getInt(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int brokerHeartbeatIntervalMs() {
+ return getInt(KRaftConfigs.BROKER_HEARTBEAT_INTERVAL_MS_CONFIG);
+ }
+
+ public final int brokerSessionTimeoutMs() {
+ return getInt(KRaftConfigs.BROKER_SESSION_TIMEOUT_MS_CONFIG);
+ }
+
+ public final int metadataLogSegmentBytes() {
+ return getInt(KRaftConfigs.METADATA_LOG_SEGMENT_BYTES_CONFIG);
+ }
+
+ public final long metadataLogSegmentMillis() {
+ return getLong(KRaftConfigs.METADATA_LOG_SEGMENT_MILLIS_CONFIG);
+ }
+
+ public final long metadataRetentionBytes() {
+ return getLong(KRaftConfigs.METADATA_MAX_RETENTION_BYTES_CONFIG);
+ }
+
+ public final long metadataRetentionMillis() {
+ return getLong(KRaftConfigs.METADATA_MAX_RETENTION_MILLIS_CONFIG);
+ }
+
+ public final int metadataNodeIDConfig() {
+ return getInt(KRaftConfigs.NODE_ID_CONFIG);
+ }
+
+ public final int metadataLogSegmentMinBytes() {
+ return getInt(KRaftConfigs.METADATA_LOG_SEGMENT_MIN_BYTES_CONFIG);
+ }
+
+ public final Boolean isKRaftCombinedMode() {
+ return processRoles().equals(new HashSet<>(Arrays.asList(ProcessRole.BrokerRole, ProcessRole.ControllerRole)));
+ }
+
+ public final String metadataLogDir() {
+ return Optional.ofNullable(getString(KRaftConfigs.METADATA_LOG_DIR_CONFIG)).orElse(logDirs().get(0));
+ }
+
+ public final boolean requiresZookeeper() {
+ return processRoles().isEmpty();
+ }
+
+ public final boolean usesSelfManagedQuorum() {
+ return !processRoles().isEmpty();
+ }
+
+ public final boolean migrationEnabled() {
+ return getBoolean(KRaftConfigs.MIGRATION_ENABLED_CONFIG);
+ }
+
+ public final int migrationMetadataMinBatchSize() {
+ return getInt(KRaftConfigs.MIGRATION_METADATA_MIN_BATCH_SIZE_CONFIG);
+ }
+
+ public final boolean elrEnabled() {
+ return getBoolean(KRaftConfigs.ELR_ENABLED_CONFIG);
+ }
+
+ public final long logRetentionTimeMillis() {
+ long millisInMinute = 60L * 1000L;
+ long millisInHour = 60L * millisInMinute;
+
+ long millis =
+ Optional.ofNullable(getLong(ServerLogConfigs.LOG_RETENTION_TIME_MILLIS_CONFIG)).orElse(
+ Optional.ofNullable(getInt(ServerLogConfigs.LOG_RETENTION_TIME_MINUTES_CONFIG))
+ .map(mins -> millisInMinute * mins)
+ .orElse(getInt(ServerLogConfigs.LOG_RETENTION_TIME_HOURS_CONFIG) * millisInHour)
+ );
+
+ if (millis < 0) return -1L;
+ return millis;
+ }
+
+ public final long serverMaxStartupTimeMs() {
+ return getLong(KRaftConfigs.SERVER_MAX_STARTUP_TIME_MS_CONFIG);
+ }
+
+ public final int numNetworkThreads() {
+ return getInt(ServerConfigs.NUM_NETWORK_THREADS_CONFIG);
+ }
+
+ public final int backgroundThreads() {
+ return getInt(ServerConfigs.BACKGROUND_THREADS_CONFIG);
+ }
+
+ public final int queuedMaxRequests() {
+ return getInt(ServerConfigs.QUEUED_MAX_REQUESTS_CONFIG);
+ }
+
+ public final long queuedMaxBytes() {
+ return getLong(ServerConfigs.QUEUED_MAX_BYTES_CONFIG);
+ }
+
+ public final int numIoThreads() {
+ return getInt(ServerConfigs.NUM_IO_THREADS_CONFIG);
+ }
+
+ public final int messageMaxBytes() {
+ return getInt(ServerConfigs.MESSAGE_MAX_BYTES_CONFIG);
+ }
+
+ public final int requestTimeoutMs() {
+ return getInt(ServerConfigs.REQUEST_TIMEOUT_MS_CONFIG);
+ }
+
+ public final long connectionSetupTimeoutMs() {
+ return getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MS_CONFIG);
+ }
+
+ public final long connectionSetupTimeoutMaxMs() {
+ return getLong(ServerConfigs.SOCKET_CONNECTION_SETUP_TIMEOUT_MAX_MS_CONFIG);
+ }
+
+ public final int getNumReplicaAlterLogDirsThreads() {
+ return Optional.ofNullable(getInt(ServerConfigs.NUM_REPLICA_ALTER_LOG_DIRS_THREADS_CONFIG)).orElse(logDirs().size());
+ }
+
+ /************* Metadata Configuration ***********/
+ public final long metadataSnapshotMaxNewRecordBytes() {
+ return getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_NEW_RECORD_BYTES_CONFIG);
+ }
+
+ public final long metadataSnapshotMaxIntervalMs() {
+ return getLong(KRaftConfigs.METADATA_SNAPSHOT_MAX_INTERVAL_MS_CONFIG);
+ }
+
+ public final Optional metadataMaxIdleIntervalNs() {
+ long value = TimeUnit.NANOSECONDS.convert((long) getInt(KRaftConfigs.METADATA_MAX_IDLE_INTERVAL_MS_CONFIG), TimeUnit.MILLISECONDS);
+ return (value > 0) ? Optional.of(value) : Optional.empty();
+ }
+
+ public final Set processRoles() {
+ return parseProcessRoles();
+ }
+
+ private Set parseProcessRoles() {
+ List rolesList = getList(KRaftConfigs.PROCESS_ROLES_CONFIG);
+ Set roles = rolesList.stream()
+ .map(role -> {
+ switch (role) {
+ case "broker":
+ return ProcessRole.BrokerRole;
+ case "controller":
+ return ProcessRole.ControllerRole;
+ default:
+ throw new ConfigException(String.format("Unknown process role '%s' " +
+ "(only 'broker' and 'controller' are allowed roles)", role));
+ }
+ })
+ .collect(Collectors.toSet());
+
+ Set distinctRoles = new HashSet<>(roles);
+
+ if (distinctRoles.size() != roles.size()) {
+ throw new ConfigException(String.format("Duplicate role names found in `%s`: %s",
+ KRaftConfigs.PROCESS_ROLES_CONFIG, roles));
+ }
+
+ return distinctRoles;
+ }
+
+ /************* Authorizer Configuration ***********/
+ public final Optional createNewAuthorizer() throws ClassNotFoundException {
+ String className = getString(ServerConfigs.AUTHORIZER_CLASS_NAME_CONFIG);
+ if (className == null || className.isEmpty())
+ return Optional.empty();
+ else {
+ return Optional.of(AuthorizerUtils.createAuthorizer(className));
+ }
+ }
+
+ public final Set earlyStartListeners() {
+ Set listenersSet = extractListenerNames();
+ Set controllerListenersSet = extractListenerNames(controllerListeners());
+ return Optional.ofNullable(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG))
+ .map(listener -> {
+ String[] listenerNames = listener.split(",");
+ Set result = new HashSet<>();
+ for (String listenerNameStr : listenerNames) {
+ String trimmedName = listenerNameStr.trim();
+ if (!trimmedName.isEmpty()) {
+ ListenerName listenerName = ListenerName.normalised(trimmedName);
+ if (!listenersSet.contains(listenerName) && !controllerListenersSet.contains(listenerName)) {
+ throw new ConfigException(
+ String.format("%s contains listener %s, but this is not contained in %s or %s",
+ ServerConfigs.EARLY_START_LISTENERS_CONFIG,
+ listenerName.value(),
+ SocketServerConfigs.LISTENERS_CONFIG,
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG));
+ }
+ result.add(listenerName);
+ }
+ }
+ return result;
+ }).orElse(controllerListenersSet);
+ }
+
+ public final List listeners() {
+ return EndpointUtils.listenerListToEndpoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap());
+ }
+
+ public final List controllerListeners() {
+ return listeners().stream()
+ .filter(l -> l.listenerName().isPresent() && controllerListenerNames().contains(l.listenerName().get()))
+ .collect(Collectors.toList());
+ }
+
+ public final List controllerListenerNames() {
+ String value = Optional.ofNullable(getString(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)).orElse("");
+ if (value.isEmpty()) {
+ return Collections.emptyList();
+ } else {
+ return Arrays.asList(value.split(","));
+ }
+ }
+
+ public final String saslMechanismControllerProtocol() {
+ return getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG);
+ }
+
+ public final Optional controlPlaneListener() {
+ return controlPlaneListenerName().flatMap(listenerName ->
+ listeners().stream()
+ .filter(endpoint -> endpoint.listenerName().isPresent())
+ .filter(endpoint -> endpoint.listenerName().get().equals(listenerName.value()))
+ .findFirst()
+ );
+ }
+
+ public final List dataPlaneListeners() {
+ return listeners().stream().filter(listener ->
+ listener.listenerName().filter(name ->
+ !name.equals(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)) ||
+ !controllerListenerNames().contains(name))
+ .isPresent()
+ ).collect(Collectors.toList());
+ }
+
+ // Topic IDs are used with all self-managed quorum clusters and ZK cluster with IBP greater than or equal to 2.8
+ public final Boolean usesTopicId() {
+ return usesSelfManagedQuorum() || interBrokerProtocolVersion().isTopicIdsSupported();
+ }
+
+ public final long logLocalRetentionBytes() {
+ return getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_BYTES_PROP);
+ }
+
+ public final long logLocalRetentionMs() {
+ return getLong(RemoteLogManagerConfig.LOG_LOCAL_RETENTION_MS_PROP);
+ }
+
+ public final int remoteFetchMaxWaitMs() {
+ return getInt(RemoteLogManagerConfig.REMOTE_FETCH_MAX_WAIT_MS_PROP);
+ }
+
+ public final long remoteLogIndexFileCacheTotalSizeBytes() {
+ return getLong(RemoteLogManagerConfig.REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES_PROP);
+ }
+
+ public final long remoteLogManagerCopyMaxBytesPerSecond() {
+ return getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_COPY_MAX_BYTES_PER_SECOND_PROP);
+ }
+
+ public final long remoteLogManagerFetchMaxBytesPerSecond() {
+ return getLong(RemoteLogManagerConfig.REMOTE_LOG_MANAGER_FETCH_MAX_BYTES_PER_SECOND_PROP);
+ }
+
+
+ // Use advertised listeners if defined, fallback to listeners otherwise
+ public final List effectiveAdvertisedListeners() {
+ String advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG);
+ if (advertisedListenersProp != null) {
+ return EndpointUtils.listenerListToEndpoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap(), false);
+ } else {
+ return listeners().stream()
+ .filter(endpoint -> endpoint.listenerName().isPresent())
+ .filter(l -> !controllerListenerNames().contains(l.listenerName().get())).collect(Collectors.toList());
+ }
+ }
+
+ protected Set extractListenerNames(Collection listeners) {
+ return listeners.stream()
+ .filter(l -> l.listenerName().isPresent())
+ .map(listener -> ListenerName.normalised(listener.listenerName().get()))
+ .collect(Collectors.toSet());
+ }
+
+ protected Set extractListenerNames() {
+ return extractListenerNames(listeners());
+ }
+
+ protected Map getMap(String propName, String propValue) {
+ try {
+ return Csv.parseCsvMap(propValue);
+ } catch (Exception e) {
+ throw new IllegalArgumentException(String.format("Error parsing configuration property '%s': %s", propName, e.getMessage()));
+ }
+ }
+
+ protected Optional> getControlPlaneListenerNameAndSecurityProtocol() {
+ return Optional.ofNullable(getString(SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG)).map(name -> {
+ ListenerName listenerName = ListenerName.normalised(name);
+ SecurityProtocol securityProtocol = Optional.ofNullable(effectiveListenerSecurityProtocolMap().get(listenerName))
+ .orElseThrow(() -> new ConfigException("Listener with " + listenerName.value() + " defined in " +
+ SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG + " not found in " + SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG + "."));
+ return new AbstractMap.SimpleEntry<>(listenerName, securityProtocol);
+ });
+ }
+
+ public final Map effectiveListenerSecurityProtocolMap() {
+ Map mapValue = getMap(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, getString(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG))
+ .entrySet()
+ .stream()
+ .collect(Collectors.toMap(entry -> ListenerName.normalised(entry.getKey()),
+ entry -> getSecurityProtocol(entry.getValue(), SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)));
+
+ if (usesSelfManagedQuorum() && !originals().containsKey(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG)) {
+ // check controller listener names (they won't appear in listeners when process.roles=broker)
+ // as well as listeners for occurrences of SSL or SASL_*
+ boolean listenerIsSslOrSasl = Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).stream()
+ .noneMatch(listenerValue -> isSslOrSasl(EndpointUtils.parseListenerName(listenerValue)));
+ if (controllerListenerNames().stream().noneMatch(this::isSslOrSasl) && listenerIsSslOrSasl) {
+ // add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT
+ mapValue.putAll(
+ controllerListenerNames().stream().filter(name -> !name.equals(SecurityProtocol.PLAINTEXT.name))
+ .collect(Collectors.toMap(ListenerName::normalised, name -> SecurityProtocol.PLAINTEXT)));
+ }
+ }
+ return mapValue; // don't add default mappings since we found something that is SSL or SASL_*
+ }
+
+ protected final Map.Entry getInterBrokerListenerNameAndSecurityProtocol() throws ConfigException {
+ Optional internalBrokerListener = Optional.ofNullable(getString(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG));
+ return internalBrokerListener.map(name -> {
+ if (originals().containsKey(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG)) {
+ throw new ConfigException("Only one of " + ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG + " and " +
+ ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG + " should be set.");
+ }
+ ListenerName listenerName = ListenerName.normalised(name);
+ SecurityProtocol securityProtocol = Optional.ofNullable(effectiveListenerSecurityProtocolMap().get(listenerName)).orElseThrow(
+ () -> new ConfigException("Listener with name " + listenerName.value() + " defined in " +
+ ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG + " not found in " + SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG + ".")
+ );
+ return new AbstractMap.SimpleEntry<>(listenerName, securityProtocol);
+ }).orElseGet(() -> {
+ SecurityProtocol securityProtocol = getSecurityProtocol(getString(ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG),
+ ReplicationConfigs.INTER_BROKER_SECURITY_PROTOCOL_CONFIG);
+ return new AbstractMap.SimpleEntry<>(ListenerName.forSecurityProtocol(securityProtocol), securityProtocol);
+ });
+ }
+
+ protected final SecurityProtocol getSecurityProtocol(String protocolName, String configName) {
+ try {
+ return SecurityProtocol.forName(protocolName);
+ } catch (IllegalArgumentException e) {
+ throw new ConfigException(String.format("Invalid security protocol `%s` defined in %s", protocolName, configName));
+ }
+ }
+
+ // Nothing was specified explicitly for listener.security.protocol.map, so we are using the default value,
+ // and we are using KRaft.
+ // Add PLAINTEXT mappings for controller listeners as long as there is no SSL or SASL_{PLAINTEXT,SSL} in use
+ private boolean isSslOrSasl(String name) {
+ return name.equals(SecurityProtocol.SSL.name) || name.equals(SecurityProtocol.SASL_SSL.name) || name.equals(SecurityProtocol.SASL_PLAINTEXT.name);
+ }
+
+ /**
+ * Copy the subset of properties that are relevant to Logs. The individual properties
+ * are listed here since the names are slightly different in each Config class...
+ */
+ @SuppressWarnings("deprecation")
+ public final Map extractLogConfigMap() {
+ return Utils.mkMap(
+ Utils.mkEntry(TopicConfig.SEGMENT_BYTES_CONFIG, logSegmentBytes()),
+ Utils.mkEntry(TopicConfig.SEGMENT_MS_CONFIG, logRollTimeMillis()),
+ Utils.mkEntry(TopicConfig.SEGMENT_JITTER_MS_CONFIG, logRollTimeJitterMillis()),
+ Utils.mkEntry(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, logIndexSizeMaxBytes()),
+ Utils.mkEntry(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, logFlushIntervalMessages()),
+ Utils.mkEntry(TopicConfig.FLUSH_MS_CONFIG, logFlushIntervalMs()),
+ Utils.mkEntry(TopicConfig.RETENTION_BYTES_CONFIG, logRetentionBytes()),
+ Utils.mkEntry(TopicConfig.RETENTION_MS_CONFIG, logRetentionTimeMillis()),
+ Utils.mkEntry(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, messageMaxBytes()),
+ Utils.mkEntry(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, logIndexIntervalBytes()),
+ Utils.mkEntry(TopicConfig.DELETE_RETENTION_MS_CONFIG, logCleanerDeleteRetentionMs()),
+ Utils.mkEntry(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, logCleanerMinCompactionLagMs()),
+ Utils.mkEntry(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, logCleanerMaxCompactionLagMs()),
+ Utils.mkEntry(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, logDeleteDelayMs()),
+ Utils.mkEntry(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, logCleanerMinCleanRatio()),
+ Utils.mkEntry(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy()),
+ Utils.mkEntry(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas()),
+ Utils.mkEntry(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType()),
+ Utils.mkEntry(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG, gzipCompressionLevel()),
+ Utils.mkEntry(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG, lz4CompressionLevel()),
+ Utils.mkEntry(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG, zstdCompressionLevel()),
+ Utils.mkEntry(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable()),
+ Utils.mkEntry(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable()),
+ Utils.mkEntry(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion().version()),
+ Utils.mkEntry(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, logMessageTimestampType().name),
+ Utils.mkEntry(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, logMessageTimestampDifferenceMaxMs()),
+ Utils.mkEntry(TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, logMessageTimestampBeforeMaxMs()),
+ Utils.mkEntry(TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, logMessageTimestampAfterMaxMs()),
+ Utils.mkEntry(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, logMessageDownConversionEnable()),
+ Utils.mkEntry(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, logLocalRetentionMs()),
+ Utils.mkEntry(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, logLocalRetentionBytes())
+ );
}
}
diff --git a/server/src/main/java/org/apache/kafka/server/config/KafkaConfigValidator.java b/server/src/main/java/org/apache/kafka/server/config/KafkaConfigValidator.java
new file mode 100644
index 0000000000000..e8a2c0b841786
--- /dev/null
+++ b/server/src/main/java/org/apache/kafka/server/config/KafkaConfigValidator.java
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.server.config;
+
+import org.slf4j.Logger;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.SaslConfigs;
+import org.apache.kafka.common.config.internals.BrokerSecurityConfigs;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.RecordVersion;
+import org.apache.kafka.common.security.auth.KafkaPrincipalSerde;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.group.GroupCoordinatorConfig;
+import org.apache.kafka.network.SocketServerConfigs;
+import org.apache.kafka.raft.QuorumConfig;
+import org.apache.kafka.server.ProcessRole;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.storage.internals.log.LogConfig;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.server.common.MetadataVersion.IBP_2_1_IV0;
+
+public class KafkaConfigValidator {
+ AbstractKafkaConfig config;
+ Logger log;
+ public KafkaConfigValidator(AbstractKafkaConfig config, Logger log) {
+ this.config = config;
+ this.log = log;
+ }
+
+ public void validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker() {
+ Set advertisedListenerNames = config.extractListenerNames(config.effectiveAdvertisedListeners());
+
+ Utils.require(advertisedListenerNames.stream().noneMatch(aln -> config.controllerListenerNames().contains(aln.value())),
+ String.format("The advertised.listeners config must not contain KRaft controller listeners from %s when %s 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.",
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, KRaftConfigs.PROCESS_ROLES_CONFIG));
+ }
+
+ public void validateControllerQuorumVotersMustContainNodeIdForKRaftController(Set voterIds) {
+ Utils.require(voterIds.contains(config.nodeId()),
+ String.format("If %s contains the 'controller' role, the node id %d must be included in the set of voters %s=%s",
+ KRaftConfigs.PROCESS_ROLES_CONFIG, config.nodeId(), QuorumConfig.QUORUM_VOTERS_CONFIG, voterIds));
+ }
+ public void validateControllerListenerExistsForKRaftController() {
+ Utils.require(!config.controllerListeners().isEmpty(),
+ String.format("%s must contain at least one value appearing in the '%s' configuration when running the KRaft controller role",
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, SocketServerConfigs.LISTENERS_CONFIG));
+ }
+ public void validateControllerListenerNamesMustAppearInListenersForKRaftController() {
+ Set listenerNameValues = config.listeners().stream()
+ .filter(l -> l.listenerName().isPresent())
+ .map(l -> l.listenerName().get())
+ .collect(Collectors.toSet());
+ Utils.require(listenerNameValues.containsAll(config.controllerListenerNames()),
+ String.format("%s must only contain values appearing in the '%s' configuration when running the KRaft controller role",
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, SocketServerConfigs.LISTENERS_CONFIG
+ ));
+ }
+ public void validateAdvertisedListenersNonEmptyForBroker() {
+ Set advertisedListenerNames = config.extractListenerNames(config.effectiveAdvertisedListeners());
+
+ Utils.require(!advertisedListenerNames.isEmpty(),
+ "There must be at least one advertised listener." + (
+ config.processRoles().contains(ProcessRole.BrokerRole) ?
+ String.format(" Perhaps all listeners appear in %s?", KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG) :
+ ""));
+ }
+ @SuppressWarnings("deprecation")
+ public String createBrokerWarningMessage() {
+ return String.format("Broker configuration %s with value %s is ignored because the inter-broker protocol version `%s` is greater or equal than 3.0. " +
+ "This configuration is deprecated and it will be removed in Apache Kafka 4.0.",
+ ServerLogConfigs.LOG_MESSAGE_FORMAT_VERSION_CONFIG, config.logMessageFormatVersionString(), config.interBrokerProtocolVersionString());
+ }
+
+ public void validateLogConfig() {
+ Utils.require(config.logRollTimeMillis() >= 1, "log.roll.ms must be greater than or equal to 1");
+ Utils.require(config.logRollTimeJitterMillis() >= 0, "log.roll.jitter.ms must be greater than or equal to 0");
+ Utils.require(config.logRetentionTimeMillis() >= 1 || config.logRetentionTimeMillis() == -1, "log.retention.ms must be unlimited (-1) or, greater than or equal to 1");
+ Utils.require(!config.logDirs().isEmpty(), "At least one log directory must be defined via log.dirs or log.dir.");
+ Utils.require(config.logCleanerDedupeBufferSize() / config.logCleanerThreads() > 1024 * 1024, "log.cleaner.dedupe.buffer.size must be at least 1MB per cleaner thread.");
+ }
+
+ private void validateNonEmptyQuorumVotersForKRaft(Set voterIds) {
+ if (voterIds.isEmpty()) {
+ throw new ConfigException(String.format("If using %s, %s must contain a parseable set of voters.", KRaftConfigs.PROCESS_ROLES_CONFIG, QuorumConfig.QUORUM_VOTERS_CONFIG));
+ }
+ }
+
+ public void validateNonEmptyQuorumVotersForMigration() {
+ if (QuorumConfig.parseVoterIds(config.quorumVoters()).isEmpty()) {
+ throw new ConfigException(String.format("If using %s, %s must contain a parseable set of voters.", KRaftConfigs.MIGRATION_ENABLED_CONFIG, QuorumConfig.QUORUM_VOTERS_CONFIG));
+ }
+ }
+
+ public void validateControlPlaneListenerEmptyForKRaft() {
+ Utils.require(!config.controlPlaneListenerName().isPresent(),
+ SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG + " is not supported in KRaft mode.");
+ }
+
+ public void validateKraftBrokerConfig() {
+ Set voterIds = QuorumConfig.parseVoterIds(config.quorumVoters());
+ // KRaft broker-only
+ validateNonEmptyQuorumVotersForKRaft(voterIds);
+ validateControlPlaneListenerEmptyForKRaft();
+ validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker();
+ // nodeId must not appear in controller.quorum.voters
+ Utils.require(!voterIds.contains(config.nodeId()),
+ String.format("If %s contains just the 'broker' role, the node id %d must not be included in the set of voters %s=%s",
+ KRaftConfigs.PROCESS_ROLES_CONFIG, config.nodeId(), QuorumConfig.QUORUM_VOTERS_CONFIG, voterIds));
+ // controller.listener.names must be non-empty...
+ Utils.require(!config.controllerListenerNames().isEmpty(),
+ String.format("%s must contain at least one value when running KRaft with just the broker role", KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG));
+ // controller.listener.names are forbidden in listeners...
+ Utils.require(config.controllerListeners().isEmpty(),
+ String.format("%s must not contain a value appearing in the '%s' configuration when running KRaft with just the broker role",
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, SocketServerConfigs.LISTENERS_CONFIG));
+ // controller.listener.names must all appear in listener.security.protocol.map
+ config.controllerListenerNames().forEach(name -> {
+ ListenerName listenerName = ListenerName.normalised(name);
+ if (!config.effectiveListenerSecurityProtocolMap().containsKey(listenerName)) {
+ throw new ConfigException(String.format("Controller listener with name %s defined in " +
+ "%s not found in %s (an explicit security mapping for each controller listener is required if %s is non-empty, or if there are security protocols other than PLAINTEXT in use)",
+ listenerName.value(), KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG));
+ }
+ });
+ // warn that only the first controller listener is used if there is more than one
+ if (config.controllerListenerNames().size() > 1) {
+ log.warn(String.format("%s has multiple entries; only the first will be used since %s=broker: %s",
+ KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG, KRaftConfigs.PROCESS_ROLES_CONFIG, config.controllerListenerNames()));
+ }
+ validateAdvertisedListenersNonEmptyForBroker();
+ }
+
+ public void validateKraftControllerConfig() {
+ // KRaft controller-only
+ Set voterIds = QuorumConfig.parseVoterIds(config.quorumVoters());
+
+ validateNonEmptyQuorumVotersForKRaft(voterIds);
+ validateControlPlaneListenerEmptyForKRaft();
+ // advertised listeners must be empty when only the controller is configured
+ Utils.require(config.getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) == null,
+ String.format("The %s config must be empty when %s=controller",
+ SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, KRaftConfigs.PROCESS_ROLES_CONFIG));
+ // listeners should only contain listeners also enumerated in the controller listener
+ Utils.require(config.effectiveAdvertisedListeners().isEmpty(),
+ String.format("The %s config must only contain KRaft controller listeners from %s when %s=controller",
+ SocketServerConfigs.LISTENERS_CONFIG, KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG,
+ KRaftConfigs.PROCESS_ROLES_CONFIG));
+ validateControllerQuorumVotersMustContainNodeIdForKRaftController(voterIds);
+ validateControllerListenerExistsForKRaftController();
+ validateControllerListenerNamesMustAppearInListenersForKRaftController();
+ }
+
+ public void validateKraftCombinedModeConfig() {
+ // KRaft combined broker and controller
+ Set voterIds = QuorumConfig.parseVoterIds(config.quorumVoters());
+
+ validateNonEmptyQuorumVotersForKRaft(voterIds);
+ validateControlPlaneListenerEmptyForKRaft();
+ validateAdvertisedListenersDoesNotContainControllerListenersForKRaftBroker();
+ validateControllerQuorumVotersMustContainNodeIdForKRaftController(voterIds);
+ validateControllerListenerExistsForKRaftController();
+ validateControllerListenerNamesMustAppearInListenersForKRaftController();
+ validateAdvertisedListenersNonEmptyForBroker();
+ }
+
+ public void validateListenerNames() {
+ Set listenerNames = config.extractListenerNames();
+ List effectiveAdvertisedListeners = config.effectiveAdvertisedListeners();
+ Set advertisedListenerNames = config.extractListenerNames(effectiveAdvertisedListeners);
+
+ Set processRoles = config.processRoles();
+ if (processRoles.isEmpty() || processRoles.contains(ProcessRole.BrokerRole)) {
+ // validations for all broker setups (i.e. ZooKeeper and KRaft broker-only and KRaft co-located)
+ validateAdvertisedListenersNonEmptyForBroker();
+ Utils.require(advertisedListenerNames.contains(config.interBrokerListenerName()),
+ String.format("%s must be a listener name defined in %s. The valid options based on currently configured listeners are %s",
+ ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG,
+ String.join(",", advertisedListenerNames.stream().map(ListenerName::value).collect(Collectors.toSet()))));
+ Utils.require(listenerNames.containsAll(advertisedListenerNames),
+ String.format("%s listener names must be equal to or a subset of the ones defined in %s. " +
+ "Found %s The valid options based on the current configuration are %s",
+ SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, SocketServerConfigs.LISTENERS_CONFIG,
+ String.join(",", advertisedListenerNames.stream().map(ListenerName::value).collect(Collectors.toSet())),
+ String.join(", ", listenerNames.stream().map(ListenerName::value).collect(Collectors.toSet()))));
+ }
+
+ Utils.require(effectiveAdvertisedListeners.stream().noneMatch(endpoint -> "0.0.0.0".equals(endpoint.host())),
+ String.format("%s cannot use the nonroutable meta-address 0.0.0.0. Use a routable IP address.",
+ SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG));
+ // validate control.plane.listener.name config
+ Optional controlPlaneListenerName = config.controlPlaneListenerName();
+ if (controlPlaneListenerName.isPresent()) {
+ Utils.require(advertisedListenerNames.contains(controlPlaneListenerName.get()),
+ String.format("%s must be a listener name defined in %s. The valid options based on currently configured listeners are %s",
+ SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG,
+ String.join(",", advertisedListenerNames.stream().map(ListenerName::value).collect(Collectors.toSet()))
+ ));
+ // controlPlaneListenerName should be different from interBrokerListenerName
+ Utils.require(!controlPlaneListenerName.get().value().equals(config.interBrokerListenerName().value()),
+ String.format("%s, when defined, should have a different value from the inter broker listener name. " +
+ "Currently they both have the value %s",
+ SocketServerConfigs.CONTROL_PLANE_LISTENER_NAME_CONFIG, controlPlaneListenerName.get()
+ ));
+ }
+ }
+
+ @SuppressWarnings("deprecation")
+ public void validateNewGroupCoordinatorConfigs() {
+ Utils.require(config.consumerGroupMaxHeartbeatIntervalMs() >= config.consumerGroupMinHeartbeatIntervalMs(),
+ String.format("%s must be greater than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG));
+ Utils.require(config.consumerGroupHeartbeatIntervalMs() >= config.consumerGroupMinHeartbeatIntervalMs(),
+ String.format("%s must be greater than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG));
+ Utils.require(config.consumerGroupHeartbeatIntervalMs() <= config.consumerGroupMaxHeartbeatIntervalMs(),
+ String.format("%s must be less than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG));
+
+ Utils.require(config.consumerGroupMaxSessionTimeoutMs() >= config.consumerGroupMinSessionTimeoutMs(),
+ String.format("%s must be greater than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG));
+ Utils.require(config.consumerGroupSessionTimeoutMs() >= config.consumerGroupMinSessionTimeoutMs(),
+ String.format("%s must be greater than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG));
+ Utils.require(config.consumerGroupSessionTimeoutMs() <= config.consumerGroupMaxSessionTimeoutMs(),
+ String.format("%s must be less than or equals to %s",
+ GroupCoordinatorConfig.CONSUMER_GROUP_SESSION_TIMEOUT_MS_CONFIG,
+ GroupCoordinatorConfig.CONSUMER_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG));
+
+ if (config.originals().containsKey(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG)) {
+ log.warn(String.format("%s is deprecated and it will be removed in Apache Kafka 4.0.", GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG));
+ }
+ }
+
+ public void validateSharedGroupConfigs() {
+ Utils.require(config.shareGroupMaxHeartbeatIntervalMs() >= config.shareGroupMinHeartbeatIntervalMs(),
+ ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG);
+ Utils.require(config.shareGroupHeartbeatIntervalMs() >= config.shareGroupMinHeartbeatIntervalMs(),
+ ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_HEARTBEAT_INTERVAL_MS_CONFIG);
+ Utils.require(config.shareGroupHeartbeatIntervalMs() <= config.shareGroupMaxHeartbeatIntervalMs(),
+ ShareGroupConfigs.SHARE_GROUP_HEARTBEAT_INTERVAL_MS_CONFIG + " must be less than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MAX_HEARTBEAT_INTERVAL_MS_CONFIG);
+
+ Utils.require(config.shareGroupMaxSessionTimeoutMs() >= config.shareGroupMinSessionTimeoutMs(),
+ ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG);
+ Utils.require(config.shareGroupSessionTimeoutMs() >= config.shareGroupMinSessionTimeoutMs(),
+ ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_SESSION_TIMEOUT_MS_CONFIG);
+ Utils.require(config.shareGroupSessionTimeoutMs() <= config.shareGroupMaxSessionTimeoutMs(),
+ ShareGroupConfigs.SHARE_GROUP_SESSION_TIMEOUT_MS_CONFIG + " must be less than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MAX_SESSION_TIMEOUT_MS_CONFIG);
+
+ Utils.require(config.shareGroupMaxRecordLockDurationMs() >= config.shareGroupMinRecordLockDurationMs(),
+ ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG);
+ Utils.require(config.shareGroupRecordLockDurationMs() >= config.shareGroupMinRecordLockDurationMs(),
+ ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_MIN_RECORD_LOCK_DURATION_MS_CONFIG);
+ Utils.require(config.shareGroupMaxRecordLockDurationMs() >= config.shareGroupRecordLockDurationMs(),
+ ShareGroupConfigs.SHARE_GROUP_MAX_RECORD_LOCK_DURATION_MS_CONFIG + " must be greater than or equals to " +
+ ShareGroupConfigs.SHARE_GROUP_RECORD_LOCK_DURATION_MS_CONFIG);
+
+ }
+
+
+ @SuppressWarnings("deprecation")
+ public void validateMessageFormatConfigs() {
+ if (new LogConfig.MessageFormatVersion(config.logMessageFormatVersionString(), config.interBrokerProtocolVersionString()).shouldWarn()) {
+ log.warn(createBrokerWarningMessage());
+ }
+
+ RecordVersion recordVersion = config.logMessageFormatVersion().highestSupportedRecordVersion();
+ Utils.require(config.interBrokerProtocolVersion().highestSupportedRecordVersion().value >= recordVersion.value,
+ String.format("log.message.format.version %s can only be used when inter.broker.protocol.version " +
+ "is set to version %s or higher",
+ config.logMessageFormatVersionString(), MetadataVersion.minSupportedFor(recordVersion).shortVersion()));
+ }
+
+ public void validateConnectionConfigs() {
+ if (config.maxConnectionsPerIp() == 0)
+ Utils.require(!config.maxConnectionsPerIpOverrides().isEmpty(), String.format("%s can be set to zero only if" +
+ " %s property is set.", SocketServerConfigs.MAX_CONNECTIONS_PER_IP_CONFIG, SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG));
+
+ Set invalidAddresses = config.maxConnectionsPerIpOverrides().keySet().stream().filter(address -> !Utils.validHostPattern(address)).collect(Collectors.toSet());
+ if (!invalidAddresses.isEmpty()) {
+ throw new IllegalArgumentException(String.format("%s contains invalid addresses : %s",
+ SocketServerConfigs.MAX_CONNECTIONS_PER_IP_OVERRIDES_CONFIG,
+ String.join(",", invalidAddresses)));
+ }
+
+ if (config.connectionsMaxIdleMs() >= 0) {
+ Utils.require(config.failedAuthenticationDelayMs() < config.connectionsMaxIdleMs(), String.format(
+ "%s=%s should always be less than %s=%s to prevent failed authentication responses from timing out",
+ SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, config.failedAuthenticationDelayMs(),
+ SocketServerConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG, config.connectionsMaxIdleMs()));
+ }
+ }
+
+ public void validateCompressionConfig() {
+ if (config.offsetsTopicCompressionType() == CompressionType.ZSTD)
+ Utils.require(config.interBrokerProtocolVersion().highestSupportedRecordVersion().value >= IBP_2_1_IV0.highestSupportedRecordVersion().value,
+ "offsets.topic.compression.codec zstd can only be used when inter.broker.protocol.version " +
+ "is set to version " + IBP_2_1_IV0.shortVersion() + " or higher");
+ }
+
+ public void validateInterBrokerSecurityConfig() {
+ Boolean interBrokerUsesSasl = config.interBrokerSecurityProtocol() == SecurityProtocol.SASL_PLAINTEXT ||
+ config.interBrokerSecurityProtocol() == SecurityProtocol.SASL_SSL;
+ Utils.require(!interBrokerUsesSasl || config.saslInterBrokerHandshakeRequestEnable() || config.saslMechanismInterBrokerProtocol() == SaslConfigs.GSSAPI_MECHANISM,
+ "Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to " + config.interBrokerProtocolVersionString());
+ Utils.require(!interBrokerUsesSasl || config.saslEnabledMechanisms(config.interBrokerListenerName()).contains(config.saslMechanismInterBrokerProtocol()),
+ String.format("%s must be included in %s when SASL is used for inter-broker communication",
+ BrokerSecurityConfigs.SASL_MECHANISM_INTER_BROKER_PROTOCOL_CONFIG, BrokerSecurityConfigs.SASL_ENABLED_MECHANISMS_CONFIG));
+
+ Class> principalBuilderClass = config.getClass(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG);
+ Utils.require(principalBuilderClass != null, BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG + " must be non-null");
+ Utils.require(KafkaPrincipalSerde.class.isAssignableFrom(principalBuilderClass),
+ BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG + " must implement KafkaPrincipalSerde");
+ }
+
+ public void validateQueueMaxByte() {
+ Utils.require(config.queuedMaxBytes() <= 0 || config.queuedMaxBytes() >= config.socketRequestMaxBytes(),
+ ServerConfigs.QUEUED_MAX_BYTES_CONFIG + " must be larger or equal to " + SocketServerConfigs.SOCKET_REQUEST_MAX_BYTES_CONFIG);
+ }
+
+ public void validateReplicaFetchConfigs() {
+ Utils.require(config.replicaFetchWaitMaxMs() <= config.replicaSocketTimeoutMs(), "replica.socket.timeout.ms should always be at least replica.fetch.wait.max.ms" +
+ " to prevent unnecessary socket timeouts");
+ Utils.require(config.replicaFetchWaitMaxMs() <= config.replicaLagTimeMaxMs(), "replica.fetch.wait.max.ms should always be less than or equal to replica.lag.time.max.ms" +
+ " to prevent frequent changes in ISR");
+ }
+
+ @SuppressWarnings("deprecation")
+ public void validateOffsetCommitAcks() {
+ Utils.require(config.offsetCommitRequiredAcks() >= -1 && config.offsetCommitRequiredAcks() <= config.offsetsTopicReplicationFactor(),
+ "offsets.commit.required.acks must be greater or equal -1 and less or equal to offsets.topic.replication.factor");
+ }
+
+ public void validateNodeAndBrokerId() {
+ if (config.nodeId() != config.brokerId()) {
+ throw new ConfigException(String.format("You must set `%s` to the same value as `%s`.",
+ KRaftConfigs.NODE_ID_CONFIG, ServerConfigs.BROKER_ID_CONFIG));
+ }
+ }
+
+ /**
+ * Validate some configurations for new MetadataVersion. A new MetadataVersion can take place when
+ * a FeatureLevelRecord for "metadata.version" is read from the cluster metadata.
+ */
+ public void validateWithMetadataVersion(MetadataVersion metadataVersion) {
+ if (config.processRoles().contains(ProcessRole.BrokerRole) && config.logDirs().size() > 1) {
+ Utils.require(metadataVersion.isDirectoryAssignmentSupported(),
+ "Multiple log directories (aka JBOD) are not supported in the current MetadataVersion " + metadataVersion + ". Need " +
+ MetadataVersion.IBP_3_7_IV2 + " or higher");
+ }
+ }
+}
diff --git a/server/src/main/java/org/apache/kafka/server/utils/EndpointUtils.java b/server/src/main/java/org/apache/kafka/server/utils/EndpointUtils.java
new file mode 100644
index 0000000000000..219b8ac2537b7
--- /dev/null
+++ b/server/src/main/java/org/apache/kafka/server/utils/EndpointUtils.java
@@ -0,0 +1,164 @@
+/*
+ * 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.utils;
+
+
+import org.apache.commons.validator.routines.InetAddressValidator;
+
+import org.apache.kafka.common.Endpoint;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.network.ListenerName;
+import org.apache.kafka.common.security.auth.SecurityProtocol;
+import org.apache.kafka.server.util.Csv;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class EndpointUtils {
+ private static InetAddressValidator inetAddressValidator = InetAddressValidator.getInstance();
+
+ private static final Map DEFAULT_SECURITY_PROTOCOL_MAP = Arrays.stream(SecurityProtocol.values())
+ .collect(Collectors.toMap(sp -> ListenerName.forSecurityProtocol(sp), sp -> sp));
+ private static Pattern uriParseExp = Pattern.compile("^(.*)://\\[?([0-9a-zA-Z\\-%._:]*)\\]?:(-?[0-9]+)");
+
+ public static List listenerListToEndpoints(String listeners, Map securityProtocolMap) {
+ return listenerListToEndpoints(listeners, securityProtocolMap, true);
+ }
+
+ public static List listenerListToEndpoints(String listeners, Map securityProtocolMap, boolean requireDistinctPorts) {
+ List endPoints;
+ try {
+ List listenerList = Csv.parseCsvList(listeners);
+ endPoints = listenerList.stream()
+ .map(listener -> createEndpoint(listener, Optional.of(securityProtocolMap)))
+ .collect(Collectors.toList());
+ } catch (Exception e) {
+ throw new IllegalArgumentException("Error creating broker listeners from '" + listeners + "': " + e.getMessage(), e);
+ }
+
+ validate(endPoints, listeners, requireDistinctPorts);
+ return endPoints;
+ }
+
+ private static void validate(List endPoints, String listeners, boolean requireDistinctPorts) {
+ List distinctListenerNames = endPoints.stream()
+ .map(Endpoint::listenerName)
+ .filter(Optional::isPresent)
+ .map(name -> new ListenerName(name.get()))
+ .distinct()
+ .collect(Collectors.toList());
+
+ if (distinctListenerNames.size() != endPoints.size()) {
+ throw new IllegalArgumentException("Each listener must have a different name, listeners: " + listeners);
+ }
+
+ Map> duplicatePorts = endPoints.stream()
+ .filter(ep -> ep.port() != 0)
+ .collect(Collectors.groupingBy(Endpoint::port));
+
+ duplicatePorts.entrySet().removeIf(entry -> entry.getValue().size() <= 1);
+
+ Map>> duplicatesPartitionedByValidIps = duplicatePorts.entrySet().stream()
+ .collect(Collectors.toMap(Map.Entry::getKey, entry ->
+ entry.getValue().stream()
+ .collect(Collectors.partitioningBy(ep ->
+ ep.host() != null && inetAddressValidator.isValid(ep.host())
+ ))
+ ));
+
+ duplicatesPartitionedByValidIps.forEach((port, partitionedDuplicates) -> {
+ List duplicatesWithIpHosts = partitionedDuplicates.getOrDefault(true, Collections.emptyList());
+ List duplicatesWithoutIpHosts = partitionedDuplicates.getOrDefault(false, Collections.emptyList());
+
+ if (requireDistinctPorts) {
+ checkDuplicateListenerPorts(duplicatesWithoutIpHosts, listeners);
+ }
+
+ if (!duplicatesWithIpHosts.isEmpty()) {
+ if (duplicatesWithIpHosts.size() == 2) {
+ if (requireDistinctPorts) {
+ String errorMessage = "If you have two listeners on " +
+ "the same port then one needs to be IPv4 and the other IPv6, listeners: " + listeners + ", port: " + port;
+ if (!validateOneIsIpv4AndOtherIpv6(duplicatesWithIpHosts.get(0).host(), duplicatesWithIpHosts.get(1).host())) {
+ throw new IllegalArgumentException(errorMessage);
+ }
+
+ if (!duplicatesWithoutIpHosts.isEmpty()) {
+ throw new IllegalArgumentException(errorMessage);
+ }
+ }
+ } else {
+ throw new IllegalArgumentException("Each listener must have a different port unless exactly one listener has " +
+ "an IPv4 address and the other IPv6 address, listeners: " + listeners + ", port: " + port);
+ }
+ }
+ });
+ }
+
+ private static boolean validateOneIsIpv4AndOtherIpv6(String first, String second) {
+ return (inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) ||
+ (inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second));
+ }
+
+ private static void checkDuplicateListenerPorts(List endpoints, String listeners) {
+ Set distinctPorts = endpoints.stream()
+ .map(Endpoint::port)
+ .collect(Collectors.toSet());
+
+ if (distinctPorts.size() != endpoints.size()) {
+ throw new IllegalArgumentException("Each listener must have a different port, listeners: " + listeners);
+ }
+ }
+
+ private static Endpoint createEndpoint(String connectionString, Optional