KAFKA-18760: Deprecate Optional<String> and return String from public Endpoint#listener (#19191)

* Deprecate org.apache.kafka.common.Endpoint#listenerName.
* Add org.apache.kafka.common.Endpoint#listener to replace
org.apache.kafka.common.Endpoint#listenerName.
* Replace org.apache.kafka.network.EndPoint with
org.apache.kafka.common.Endpoint.
* Deprecate org.apache.kafka.clients.admin.RaftVoterEndpoint#name
* Add org.apache.kafka.clients.admin.RaftVoterEndpoint#listener to
replace org.apache.kafka.clients.admin.RaftVoterEndpoint#name

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, TaiJuWu
 <tjwu1217@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>, TengYao
 Chi <frankvicky@apache.org>, Ken Huang <s7133700@gmail.com>, Bagda
 Parth  , Kuan-Po Tseng <brandboat@gmail.com>

---------

Signed-off-by: PoAn Yang <payang@apache.org>
This commit is contained in:
PoAn Yang 2025-04-30 12:15:33 +08:00 committed by GitHub
parent 676e0f2ad6
commit 81881dee83
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
30 changed files with 179 additions and 200 deletions

View File

@ -4948,7 +4948,7 @@ public class KafkaAdminClient extends AdminClient {
new AddRaftVoterRequestData.ListenerCollection();
endpoints.forEach(endpoint ->
listeners.add(new AddRaftVoterRequestData.Listener().
setName(endpoint.name()).
setName(endpoint.listener()).
setHost(endpoint.host()).
setPort(endpoint.port())));
return new AddRaftVoterRequest.Builder(

View File

@ -26,7 +26,7 @@ import java.util.Objects;
*/
@InterfaceStability.Stable
public class RaftVoterEndpoint {
private final String name;
private final String listener;
private final String host;
private final int port;
@ -49,22 +49,33 @@ public class RaftVoterEndpoint {
/**
* Create an endpoint for a metadata quorum voter.
*
* @param name The human-readable name for this endpoint. For example, CONTROLLER.
* @param listener The human-readable name for this endpoint. For example, CONTROLLER.
* @param host The DNS hostname for this endpoint.
* @param port The network port for this endpoint.
*/
public RaftVoterEndpoint(
String name,
String listener,
String host,
int port
) {
this.name = requireNonNullAllCapsNonEmpty(name);
this.listener = requireNonNullAllCapsNonEmpty(listener);
this.host = Objects.requireNonNull(host);
this.port = port;
}
/**
* The listener name for this endpoint.
*/
public String listener() {
return listener;
}
/**
* @deprecated Since 4.1. Use {@link #listener()} instead. This function will be removed in 5.0.
*/
@Deprecated(since = "4.1", forRemoval = true)
public String name() {
return name;
return listener;
}
public String host() {
@ -79,20 +90,20 @@ public class RaftVoterEndpoint {
public boolean equals(Object o) {
if (o == null || (!o.getClass().equals(getClass()))) return false;
RaftVoterEndpoint other = (RaftVoterEndpoint) o;
return name.equals(other.name) &&
return listener.equals(other.listener) &&
host.equals(other.host) &&
port == other.port;
}
@Override
public int hashCode() {
return Objects.hash(name, host, port);
return Objects.hash(listener, host, port);
}
@Override
public String toString() {
// enclose IPv6 hosts in square brackets for readability
String hostString = host.contains(":") ? "[" + host + "]" : host;
return name + "://" + hostString + ":" + port;
return listener + "://" + hostString + ":" + port;
}
}

View File

@ -16,7 +16,6 @@
*/
package org.apache.kafka.common;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import java.util.Objects;
@ -26,27 +25,35 @@ import java.util.Optional;
* Represents a broker endpoint.
*/
@InterfaceStability.Evolving
public class Endpoint {
private final String listenerName;
private final String listener;
private final SecurityProtocol securityProtocol;
private final String host;
private final int port;
public Endpoint(String listenerName, SecurityProtocol securityProtocol, String host, int port) {
this.listenerName = listenerName;
public Endpoint(String listener, SecurityProtocol securityProtocol, String host, int port) {
this.listener = listener;
this.securityProtocol = securityProtocol;
this.host = host;
this.port = port;
}
/**
* Returns the listener name of this endpoint.
*/
public String listener() {
return listener;
}
/**
* Returns the listener name of this endpoint. This is non-empty for endpoints provided
* to broker plugins, but may be empty when used in clients.
* @deprecated Since 4.1. Use {@link #listener()} instead. This function will be removed in 5.0.
*/
@Deprecated(since = "4.1", forRemoval = true)
public Optional<String> listenerName() {
return Optional.ofNullable(listenerName);
return Optional.ofNullable(listener);
}
/**
@ -80,7 +87,7 @@ public class Endpoint {
}
Endpoint that = (Endpoint) o;
return Objects.equals(this.listenerName, that.listenerName) &&
return Objects.equals(this.listener, that.listener) &&
Objects.equals(this.securityProtocol, that.securityProtocol) &&
Objects.equals(this.host, that.host) &&
this.port == that.port;
@ -89,13 +96,13 @@ public class Endpoint {
@Override
public int hashCode() {
return Objects.hash(listenerName, securityProtocol, host, port);
return Objects.hash(listener, securityProtocol, host, port);
}
@Override
public String toString() {
return "Endpoint(" +
"listenerName='" + listenerName + '\'' +
"listenerName='" + listener + '\'' +
", securityProtocol=" + securityProtocol +
", host='" + host + '\'' +
", port=" + port +

View File

@ -29,7 +29,6 @@ import kafka.network.Processor._
import kafka.network.RequestChannel.{CloseConnectionResponse, EndThrottlingResponse, NoOpResponse, SendResponse, StartThrottlingResponse}
import kafka.network.SocketServer._
import kafka.server.{BrokerReconfigurable, KafkaConfig}
import org.apache.kafka.network.EndPoint
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import kafka.utils._
import org.apache.kafka.common.config.ConfigException
@ -96,7 +95,7 @@ class SocketServer(
memoryPoolSensor.add(new Meter(TimeUnit.MILLISECONDS, memoryPoolDepletedPercentMetricName, memoryPoolDepletedTimeMetricName))
private val memoryPool = if (config.queuedMaxBytes > 0) new SimpleMemoryPool(config.queuedMaxBytes, config.socketRequestMaxBytes, false, memoryPoolSensor) else MemoryPool.NONE
// data-plane
private[network] val dataPlaneAcceptors = new ConcurrentHashMap[EndPoint, DataPlaneAcceptor]()
private[network] val dataPlaneAcceptors = new ConcurrentHashMap[Endpoint, DataPlaneAcceptor]()
val dataPlaneRequestChannel = new RequestChannel(maxQueuedRequests, time, apiVersionManager.newRequestMetrics)
private[this] val nextProcessorId: AtomicInteger = new AtomicInteger(0)
@ -161,8 +160,8 @@ class SocketServer(
* Therefore, we do not know that any particular request processor will be running by the end of
* this function -- just that it might be running.
*
* @param authorizerFutures Future per [[EndPoint]] used to wait before starting the
* processor corresponding to the [[EndPoint]]. Any endpoint
* @param authorizerFutures Future per [[Endpoint]] used to wait before starting the
* processor corresponding to the [[Endpoint]]. Any endpoint
* that does not appear in this map will be started once all
* authorizerFutures are complete.
*
@ -181,7 +180,7 @@ class SocketServer(
// Because of ephemeral ports, we need to match acceptors to futures by looking at
// the listener name, rather than the endpoint object.
val authorizerFuture = authorizerFutures.find {
case (endpoint, _) => acceptor.endPoint.listenerName.value().equals(endpoint.listenerName().get())
case (endpoint, _) => acceptor.endPoint.listener.equals(endpoint.listener())
} match {
case None => allAuthorizerFuturesComplete
case Some((_, future)) => future
@ -210,23 +209,24 @@ class SocketServer(
enableFuture
}
private def createDataPlaneAcceptorAndProcessors(endpoint: EndPoint): Unit = synchronized {
private def createDataPlaneAcceptorAndProcessors(endpoint: Endpoint): Unit = synchronized {
if (stopped) {
throw new RuntimeException("Can't create new data plane acceptor and processors: SocketServer is stopped.")
}
val parsedConfigs = config.valuesFromThisConfigWithPrefixOverride(endpoint.listenerName.configPrefix)
connectionQuotas.addListener(config, endpoint.listenerName)
val isPrivilegedListener = config.interBrokerListenerName == endpoint.listenerName
val listenerName = ListenerName.normalised(endpoint.listener)
val parsedConfigs = config.valuesFromThisConfigWithPrefixOverride(listenerName.configPrefix)
connectionQuotas.addListener(config, listenerName)
val isPrivilegedListener = config.interBrokerListenerName == listenerName
val dataPlaneAcceptor = createDataPlaneAcceptor(endpoint, isPrivilegedListener, dataPlaneRequestChannel)
config.addReconfigurable(dataPlaneAcceptor)
dataPlaneAcceptor.configure(parsedConfigs)
dataPlaneAcceptors.put(endpoint, dataPlaneAcceptor)
info(s"Created data-plane acceptor and processors for endpoint : ${endpoint.listenerName}")
info(s"Created data-plane acceptor and processors for endpoint : ${listenerName}")
}
private def endpoints = config.listeners.map(l => l.listenerName -> l).toMap
private def endpoints = config.listeners.map(l => ListenerName.normalised(l.listener) -> l).toMap
protected def createDataPlaneAcceptor(endPoint: EndPoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel): DataPlaneAcceptor = {
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)
}
@ -277,7 +277,7 @@ class SocketServer(
/**
* 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.")
}
@ -297,10 +297,10 @@ class SocketServer(
}
}
def removeListeners(listenersRemoved: Seq[EndPoint]): Unit = synchronized {
def removeListeners(listenersRemoved: Seq[Endpoint]): Unit = synchronized {
info(s"Removing data-plane listeners for endpoints $listenersRemoved")
listenersRemoved.foreach { endpoint =>
connectionQuotas.removeListener(config, endpoint.listenerName)
connectionQuotas.removeListener(config, ListenerName.normalised(endpoint.listener))
dataPlaneAcceptors.asScala.remove(endpoint).foreach { acceptor =>
acceptor.beginShutdown()
acceptor.close()
@ -345,7 +345,7 @@ class SocketServer(
// For test usage
def dataPlaneAcceptor(listenerName: String): Option[DataPlaneAcceptor] = {
dataPlaneAcceptors.asScala.foreach { case (endPoint, acceptor) =>
if (endPoint.listenerName.value() == listenerName)
if (endPoint.listener == listenerName)
return Some(acceptor)
}
None
@ -374,7 +374,7 @@ object DataPlaneAcceptor {
}
class DataPlaneAcceptor(socketServer: SocketServer,
endPoint: EndPoint,
endPoint: Endpoint,
config: KafkaConfig,
nodeId: Int,
connectionQuotas: ConnectionQuotas,
@ -404,7 +404,7 @@ class DataPlaneAcceptor(socketServer: SocketServer,
* Returns the listener name associated with this reconfigurable. Listener-specific
* configs corresponding to this listener name are provided for reconfiguration.
*/
override def listenerName(): ListenerName = endPoint.listenerName
override def listenerName(): ListenerName = ListenerName.normalised(endPoint.listener)
/**
* Returns the names of configs that may be reconfigured.
@ -451,7 +451,7 @@ class DataPlaneAcceptor(socketServer: SocketServer,
val newNumNetworkThreads = configs.get(SocketServerConfigs.NUM_NETWORK_THREADS_CONFIG).asInstanceOf[Int]
if (newNumNetworkThreads != processors.length) {
info(s"Resizing network thread pool size for ${endPoint.listenerName} listener from ${processors.length} to $newNumNetworkThreads")
info(s"Resizing network thread pool size for ${endPoint.listener} listener from ${processors.length} to $newNumNetworkThreads")
if (newNumNetworkThreads > processors.length) {
addProcessors(newNumNetworkThreads - processors.length)
} else if (newNumNetworkThreads < processors.length) {
@ -472,7 +472,7 @@ class DataPlaneAcceptor(socketServer: SocketServer,
* Thread that accepts and configures new connections. There is one of these per endpoint.
*/
private[kafka] abstract class Acceptor(val socketServer: SocketServer,
val endPoint: EndPoint,
val endPoint: Endpoint,
var config: KafkaConfig,
nodeId: Int,
val connectionQuotas: ConnectionQuotas,
@ -515,7 +515,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
private val backwardCompatibilityMetricGroup = new KafkaMetricsGroup("kafka.network", "Acceptor")
private val blockedPercentMeterMetricName = backwardCompatibilityMetricGroup.metricName(
"AcceptorBlockedPercent",
Map(ListenerMetricTag -> endPoint.listenerName.value).asJava)
Map(ListenerMetricTag -> endPoint.listener).asJava)
private val blockedPercentMeter = metricsGroup.newMeter(blockedPercentMeterMetricName,"blocked time", TimeUnit.NANOSECONDS)
private var currentProcessorIndex = 0
private[network] val throttledSockets = new mutable.PriorityQueue[DelayedCloseSocket]()
@ -523,7 +523,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
private[network] val startedFuture = new CompletableFuture[Void]()
val thread: KafkaThread = KafkaThread.nonDaemon(
s"data-plane-kafka-socket-acceptor-${endPoint.listenerName}-${endPoint.securityProtocol}-${endPoint.port}",
s"data-plane-kafka-socket-acceptor-${endPoint.listener}-${endPoint.securityProtocol}-${endPoint.port}",
this)
def start(): Unit = synchronized {
@ -535,19 +535,19 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
serverChannel = openServerSocket(endPoint.host, endPoint.port, listenBacklogSize)
debug(s"Opened endpoint ${endPoint.host}:${endPoint.port}")
}
debug(s"Starting processors for listener ${endPoint.listenerName}")
debug(s"Starting processors for listener ${endPoint.listener}")
processors.foreach(_.start())
debug(s"Starting acceptor thread for listener ${endPoint.listenerName}")
debug(s"Starting acceptor thread for listener ${endPoint.listener}")
thread.start()
startedFuture.complete(null)
started.set(true)
} catch {
case e: ClosedChannelException =>
debug(s"Refusing to start acceptor for ${endPoint.listenerName} since the acceptor has already been shut down.")
debug(s"Refusing to start acceptor for ${endPoint.listener} since the acceptor has already been shut down.")
startedFuture.completeExceptionally(e)
case t: Throwable =>
error(s"Unable to start acceptor for ${endPoint.listenerName}", t)
startedFuture.completeExceptionally(new RuntimeException(s"Unable to start acceptor for ${endPoint.listenerName}", t))
error(s"Unable to start acceptor for ${endPoint.listener}", t)
startedFuture.completeExceptionally(new RuntimeException(s"Unable to start acceptor for ${endPoint.listener}", t))
}
}
@ -628,7 +628,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
new InetSocketAddress(host, port)
}
val serverChannel = socketServer.socketFactory.openServerSocket(
endPoint.listenerName.value(),
endPoint.listener,
socketAddress,
listenBacklogSize,
recvBufferSize)
@ -682,14 +682,15 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
private def accept(key: SelectionKey): Option[SocketChannel] = {
val serverSocketChannel = key.channel().asInstanceOf[ServerSocketChannel]
val socketChannel = serverSocketChannel.accept()
val listenerName = ListenerName.normalised(endPoint.listener)
try {
connectionQuotas.inc(endPoint.listenerName, socketChannel.socket.getInetAddress, blockedPercentMeter)
connectionQuotas.inc(listenerName, socketChannel.socket.getInetAddress, blockedPercentMeter)
configureAcceptedSocketChannel(socketChannel)
Some(socketChannel)
} catch {
case e: TooManyConnectionsException =>
info(s"Rejected connection from ${e.ip}, address already has the configured maximum of ${e.count} connections.")
connectionQuotas.closeChannel(this, endPoint.listenerName, socketChannel)
connectionQuotas.closeChannel(this, listenerName, socketChannel)
None
case e: ConnectionThrottledException =>
val ip = socketChannel.socket.getInetAddress
@ -699,7 +700,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
None
case e: IOException =>
error(s"Encountered an error while configuring the connection, closing it.", e)
connectionQuotas.closeChannel(this, endPoint.listenerName, socketChannel)
connectionQuotas.closeChannel(this, listenerName, socketChannel)
None
}
}
@ -741,7 +742,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
def wakeup(): Unit = nioSelector.wakeup()
def addProcessors(toCreate: Int): Unit = synchronized {
val listenerName = endPoint.listenerName
val listenerName = ListenerName.normalised(endPoint.listener)
val securityProtocol = endPoint.securityProtocol
val listenerProcessors = new ArrayBuffer[Processor]()
@ -761,7 +762,7 @@ private[kafka] abstract class Acceptor(val socketServer: SocketServer,
listenerName: ListenerName,
securityProtocol: SecurityProtocol,
connectionDisconnectListeners: Seq[ConnectionDisconnectListener]): Processor = {
val name = s"data-plane-kafka-network-thread-$nodeId-${endPoint.listenerName}-${endPoint.securityProtocol}-$id"
val name = s"data-plane-kafka-network-thread-$nodeId-${endPoint.listener}-${endPoint.securityProtocol}-$id"
new Processor(id,
time,
config.socketRequestMaxBytes,

View File

@ -274,7 +274,7 @@ class BrokerServer(
clientQuotaMetadataManager = new ClientQuotaMetadataManager(quotaManagers, socketServer.connectionQuotas)
val listenerInfo = ListenerInfo.create(Optional.of(config.interBrokerListenerName.value()),
config.effectiveAdvertisedBrokerListeners.map(_.toPublic()).asJava).
config.effectiveAdvertisedBrokerListeners.asJava).
withWildcardHostnamesResolved().
withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))

View File

@ -174,7 +174,7 @@ class ControllerServer(
sharedServer.socketFactory)
val listenerInfo = ListenerInfo
.create(config.effectiveAdvertisedControllerListeners.map(_.toPublic).asJava)
.create(config.effectiveAdvertisedControllerListeners.asJava)
.withWildcardHostnamesResolved()
.withEphemeralPortsCorrected(name => socketServer.boundPort(new ListenerName(name)))
socketServerFirstBoundPortFuture.complete(listenerInfo.firstListener().port())

View File

@ -27,7 +27,7 @@ import kafka.raft.KafkaRaftManager
import kafka.server.DynamicBrokerConfig._
import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.network.EndPoint
import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SslConfigs}
import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType}
@ -961,9 +961,9 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
def validateReconfiguration(newConfig: KafkaConfig): Unit = {
val oldConfig = server.config
val newListeners = newConfig.listeners.map(_.listenerName).toSet
val oldAdvertisedListeners = oldConfig.effectiveAdvertisedBrokerListeners.map(_.listenerName).toSet
val oldListeners = oldConfig.listeners.map(_.listenerName).toSet
val newListeners = newConfig.listeners.map(l => ListenerName.normalised(l.listener)).toSet
val oldAdvertisedListeners = oldConfig.effectiveAdvertisedBrokerListeners.map(l => ListenerName.normalised(l.listener)).toSet
val oldListeners = oldConfig.listeners.map(l => ListenerName.normalised(l.listener)).toSet
if (!oldAdvertisedListeners.subsetOf(newListeners))
throw new ConfigException(s"Advertised listeners '$oldAdvertisedListeners' must be a subset of listeners '$newListeners'")
if (!newListeners.subsetOf(newConfig.effectiveListenerSecurityProtocolMap.keySet))
@ -988,8 +988,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.filterNot(e => newListenerMap.contains(ListenerName.normalised(e.listener)))
val listenersAdded = newListeners.filterNot(e => oldListenerMap.contains(ListenerName.normalised(e.listener)))
if (listenersRemoved.nonEmpty || listenersAdded.nonEmpty) {
LoginManager.closeAll() // Clear SASL login cache to force re-login
if (listenersRemoved.nonEmpty) server.socketServer.removeListeners(listenersRemoved)
@ -997,8 +997,8 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
}
}
private def listenersToMap(listeners: Seq[EndPoint]): Map[ListenerName, EndPoint] =
listeners.map(e => (e.listenerName, e)).toMap
private def listenersToMap(listeners: Seq[Endpoint]): Map[ListenerName, Endpoint] =
listeners.map(e => (ListenerName.normalised(e.listener), e)).toMap
}

View File

@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
import java.util.Properties
import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._
import org.apache.kafka.common.Reconfigurable
import org.apache.kafka.common.{Endpoint, KafkaException, Reconfigurable}
import org.apache.kafka.common.config.{ConfigDef, ConfigException, ConfigResource, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs
@ -34,7 +34,6 @@ import org.apache.kafka.common.record.TimestampType
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.network.EndPoint
import org.apache.kafka.coordinator.group.Group.GroupType
import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig
import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig}
@ -145,6 +144,14 @@ object KafkaConfig {
}
output
}
private def parseListenerName(connectionString: String): String = {
val firstColon = connectionString.indexOf(':')
if (firstColon < 0) {
throw new KafkaException(s"Unable to parse a listener name from $connectionString")
}
connectionString.substring(0, firstColon).toUpperCase(util.Locale.ROOT)
}
}
/**
@ -276,8 +283,8 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
}
val earlyStartListeners: Set[ListenerName] = {
val listenersSet = listeners.map(_.listenerName).toSet
val controllerListenersSet = controllerListeners.map(_.listenerName).toSet
val listenersSet = listeners.map(l => ListenerName.normalised(l.listener)).toSet
val controllerListenersSet = controllerListeners.map(l => ListenerName.normalised(l.listener)).toSet
Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match {
case None => controllerListenersSet
case Some(str) =>
@ -459,7 +466,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
}
}
def listeners: Seq[EndPoint] =
def listeners: Seq[Endpoint] =
CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
def controllerListenerNames: Seq[String] = {
@ -471,23 +478,23 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
}
}
def controllerListeners: Seq[EndPoint] =
listeners.filter(l => controllerListenerNames.contains(l.listenerName.value()))
def controllerListeners: Seq[Endpoint] =
listeners.filter(l => controllerListenerNames.contains(l.listener))
def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG)
def dataPlaneListeners: Seq[EndPoint] = {
def dataPlaneListeners: Seq[Endpoint] = {
listeners.filterNot { listener =>
val name = listener.listenerName.value()
val name = listener.listener
controllerListenerNames.contains(name)
}
}
def effectiveAdvertisedControllerListeners: Seq[EndPoint] = {
def effectiveAdvertisedControllerListeners: Seq[Endpoint] = {
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
val controllerAdvertisedListeners = if (advertisedListenersProp != null) {
CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
.filter(l => controllerListenerNames.contains(l.listenerName.value()))
.filter(l => controllerListenerNames.contains(l.listener))
} else {
Seq.empty
}
@ -495,16 +502,16 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
controllerListenerNames.flatMap { name =>
controllerAdvertisedListeners
.find(endpoint => endpoint.listenerName.equals(ListenerName.normalised(name)))
.find(endpoint => ListenerName.normalised(endpoint.listener).equals(ListenerName.normalised(name)))
.orElse(
// If users don't define advertised.listeners, the advertised controller listeners inherit from listeners configuration
// which match listener names in controller.listener.names.
// Removing "0.0.0.0" host to avoid validation errors. This is to be compatible with the old behavior before 3.9.
// The null or "" host does a reverse lookup in ListenerInfo#withWildcardHostnamesResolved.
controllerListenersValue
.find(endpoint => endpoint.listenerName.equals(ListenerName.normalised(name)))
.find(endpoint => ListenerName.normalised(endpoint.listener).equals(ListenerName.normalised(name)))
.map(endpoint => if (endpoint.host == "0.0.0.0") {
new EndPoint(null, endpoint.port, endpoint.listenerName, endpoint.securityProtocol)
new Endpoint(endpoint.listener, endpoint.securityProtocol, null, endpoint.port)
} else {
endpoint
})
@ -512,7 +519,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
}
}
def effectiveAdvertisedBrokerListeners: Seq[EndPoint] = {
def effectiveAdvertisedBrokerListeners: Seq[Endpoint] = {
// Use advertised listeners if defined, fallback to listeners otherwise
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
val advertisedListeners = if (advertisedListenersProp != null) {
@ -521,7 +528,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
listeners
}
// Only expose broker listeners
advertisedListeners.filterNot(l => controllerListenerNames.contains(l.listenerName.value()))
advertisedListeners.filterNot(l => controllerListenerNames.contains(l.listener))
}
private def getInterBrokerListenerNameAndSecurityProtocol: (ListenerName, SecurityProtocol) = {
@ -563,7 +570,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
// 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)))) {
Csv.parseCsvList(getString(SocketServerConfigs.LISTENERS_CONFIG)).asScala.exists(listenerValue => isSslOrSasl(KafkaConfig.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
@ -591,7 +598,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
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")
val advertisedBrokerListenerNames = effectiveAdvertisedBrokerListeners.map(_.listenerName).toSet
val advertisedBrokerListenerNames = effectiveAdvertisedBrokerListeners.map(l => ListenerName.normalised(l.listener)).toSet
// validate KRaft-related configs
val voterIds = QuorumConfig.parseVoterIds(quorumConfig.voters)
@ -614,7 +621,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
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
val listenerNameValues = listeners.map(_.listener).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")
}
@ -681,7 +688,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
validateControllerListenerNamesMustAppearInListenersForKRaftController()
}
val listenerNames = listeners.map(_.listenerName).toSet
val listenerNames = listeners.map(l => ListenerName.normalised(l.listener)).toSet
if (processRoles.contains(ProcessRole.BrokerRole)) {
validateAdvertisedBrokerListenersNonEmptyForBroker()
require(advertisedBrokerListenerNames.contains(interBrokerListenerName),

View File

@ -30,6 +30,7 @@ import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing
import org.apache.kafka.common.metrics.stats.{Meter, Percentile, Percentiles}
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable}
import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
@ -88,7 +89,7 @@ class TestRaftServer(
val endpoints = Endpoints.fromInetSocketAddresses(
config.effectiveAdvertisedControllerListeners
.map { endpoint =>
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
(ListenerName.normalised(endpoint.listener), InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
}
.toMap
.asJava

View File

@ -25,8 +25,8 @@ import com.typesafe.scalalogging.Logger
import javax.management.ObjectName
import scala.collection._
import scala.collection.Seq
import org.apache.kafka.network.EndPoint
import org.apache.commons.validator.routines.InetAddressValidator
import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils
@ -122,22 +122,22 @@ object CoreUtils {
def inWriteLock[T](lock: ReadWriteLock)(fun: => T): T = inLock[T](lock.writeLock)(fun)
def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol]): Seq[EndPoint] = {
def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol]): Seq[Endpoint] = {
listenerListToEndPoints(listeners, securityProtocolMap, requireDistinctPorts = true)
}
private def checkDuplicateListenerPorts(endpoints: Seq[EndPoint], listeners: String): Unit = {
private def checkDuplicateListenerPorts(endpoints: Seq[Endpoint], listeners: String): Unit = {
val distinctPorts = endpoints.map(_.port).distinct
require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: $listeners")
}
def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[EndPoint] = {
def listenerListToEndPoints(listeners: String, securityProtocolMap: Map[ListenerName, SecurityProtocol], requireDistinctPorts: Boolean): Seq[Endpoint] = {
def validateOneIsIpv4AndOtherIpv6(first: String, second: String): Boolean =
(inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) ||
(inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second))
def validate(endPoints: Seq[EndPoint]): Unit = {
val distinctListenerNames = endPoints.map(_.listenerName).distinct
def validate(endPoints: Seq[Endpoint]): Unit = {
val distinctListenerNames = endPoints.map(_.listener).distinct
require(distinctListenerNames.size == endPoints.size, s"Each listener must have a different name, listeners: $listeners")
val (duplicatePorts, _) = endPoints.filter {
@ -186,8 +186,7 @@ object CoreUtils {
}
val endPoints = try {
SocketServerConfigs.listenerListToEndPoints(listeners, securityProtocolMap.asJava).
asScala.map(EndPoint.fromPublic)
SocketServerConfigs.listenerListToEndPoints(listeners, securityProtocolMap.asJava).asScala
} catch {
case e: Exception =>
throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e)

View File

@ -353,7 +353,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest {
val controllerListenerName = ListenerName.forSecurityProtocol(extraControllerSecurityProtocol)
val config = controllerServers.map { s =>
val listener = s.config.effectiveAdvertisedControllerListeners
.find(_.listenerName == controllerListenerName)
.find(_.listener == controllerListenerName.value)
.getOrElse(throw new IllegalArgumentException(s"Could not find listener with name $controllerListenerName"))
Utils.formatAddress(listener.host, s.socketServer.boundPort(controllerListenerName))
}.mkString(",")

View File

@ -113,7 +113,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(ListenerName.normalised(controller.config.controllerListeners.head.listener))
// shutdown active controller
controller.shutdown()

View File

@ -132,7 +132,7 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
TestUtils.ensureConsistentKRaftMetadata(servers, controllerServer)
servers.head.config.listeners.foreach { endPoint =>
val listenerName = endPoint.listenerName
val listenerName = ListenerName.normalised(endPoint.listener)
val trustStoreFile =
if (JaasTestUtils.usesSslTransportLayer(endPoint.securityProtocol)) Some(this.trustStoreFile)
@ -155,7 +155,7 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
}
if (JaasTestUtils.usesSaslAuthentication(endPoint.securityProtocol)) {
kafkaServerSaslMechanisms(endPoint.listenerName.value).foreach { mechanism =>
kafkaServerSaslMechanisms(endPoint.listener).foreach { mechanism =>
addProducerConsumer(listenerName, mechanism, Some(kafkaClientSaslProperties(mechanism, dynamicJaasConfig = true)))
}
} else {

View File

@ -22,6 +22,7 @@ import com.yammer.metrics.core.{Gauge, Meter}
import kafka.server._
import kafka.utils.Implicits._
import kafka.utils.TestUtils
import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.message.{ProduceRequestData, SaslAuthenticateRequestData, SaslHandshakeRequestData, VoteRequestData}
@ -36,7 +37,6 @@ import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.utils._
import org.apache.kafka.network.RequestConvertToJson
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.EndPoint
import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.{ApiVersionManager, SimpleApiVersionManager}
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
@ -91,10 +91,10 @@ class SocketServerTest {
private val kafkaLogger = LogManager.getLogger("kafka")
private var logLevelToRestore: Level = _
def endpoint: EndPoint = {
def endpoint: Endpoint = {
KafkaConfig.fromProps(props, doLog = false).dataPlaneListeners.head
}
def listener: String = endpoint.listenerName.value
def listener: String = endpoint.listener
val uncaughtExceptions = new AtomicInteger(0)
@BeforeEach
@ -840,7 +840,7 @@ class SocketServerTest {
// same as SocketServer.createAcceptor,
// except the Acceptor overriding a method to inject the exception
override protected def createDataPlaneAcceptor(endPoint: EndPoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel): DataPlaneAcceptor = {
override protected def createDataPlaneAcceptor(endPoint: Endpoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel): DataPlaneAcceptor = {
new DataPlaneAcceptor(this, endPoint, this.config, nodeId, connectionQuotas, time, false, requestChannel, serverMetrics, this.credentialProvider, new LogContext(), MemoryPool.NONE, this.apiVersionManager) {
override protected def configureAcceptedSocketChannel(socketChannel: SocketChannel): Unit = {
@ -1858,7 +1858,7 @@ class SocketServerTest {
val failedFuture = new CompletableFuture[Void]()
failedFuture.completeExceptionally(new RuntimeException("authorizer startup failed"))
assertThrows(classOf[ExecutionException], () => {
newServer.enableRequestProcessing(Map(endpoint.toPublic -> failedFuture)).get()
newServer.enableRequestProcessing(Map(endpoint -> failedFuture)).get()
})
} finally {
shutdownServerAndMetrics(newServer)
@ -1891,7 +1891,7 @@ class SocketServerTest {
val authorizerFuture = new CompletableFuture[Void]()
val enableFuture = newServer.enableRequestProcessing(
newServer.dataPlaneAcceptors.keys().asScala.
map(_.toPublic).map(k => k -> authorizerFuture).toMap)
map(k => k -> authorizerFuture).toMap)
assertFalse(authorizerFuture.isDone)
assertFalse(enableFuture.isDone)
newServer.dataPlaneAcceptors.values().forEach(a => assertNull(a.serverChannel))
@ -1992,7 +1992,7 @@ class SocketServerTest {
}
class TestableAcceptor(socketServer: SocketServer,
endPoint: EndPoint,
endPoint: Endpoint,
cfg: KafkaConfig,
nodeId: Int,
connectionQuotas: ConnectionQuotas,
@ -2098,7 +2098,7 @@ class SocketServerTest {
connectionDisconnectListeners = connectionDisconnectListeners
) {
override def createDataPlaneAcceptor(endPoint: EndPoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel) : DataPlaneAcceptor = {
override def createDataPlaneAcceptor(endPoint: Endpoint, isPrivilegedListener: Boolean, requestChannel: RequestChannel) : DataPlaneAcceptor = {
new TestableAcceptor(this, endPoint, this.config, 0, connectionQuotas, time, isPrivilegedListener, requestChannel, this.metrics, this.credentialProvider, new LogContext, MemoryPool.NONE, this.apiVersionManager, connectionQueueSize)
}

View File

@ -28,6 +28,7 @@ import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.network.ListenerName
import org.apache.kafka.common.utils.Time
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig}
@ -88,7 +89,7 @@ class RaftManagerTest {
val endpoints = Endpoints.fromInetSocketAddresses(
config.effectiveAdvertisedControllerListeners
.map { endpoint =>
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
(ListenerName.normalised(endpoint.listener), InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
}
.toMap
.asJava

View File

@ -76,7 +76,7 @@ class ControllerRegistrationManagerTest {
"controller-registration-manager-test-",
createSupportedFeatures(MetadataVersion.IBP_3_7_IV0),
RecordTestUtils.createTestControllerRegistration(1, false).incarnationId(),
ListenerInfo.create(context.config.controllerListeners.map(_.toPublic).asJava),
ListenerInfo.create(context.config.controllerListeners.asJava),
new ExponentialBackoff(1, 2, 100, 0.02))
}

View File

@ -1036,7 +1036,7 @@ class DynamicBrokerConfigTest {
props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "SASL_PLAINTEXT://localhost:8181")
ctx.config.dynamicConfig.updateDefaultConfig(props)
ctx.config.effectiveAdvertisedBrokerListeners.foreach(e =>
assertEquals(SecurityProtocol.PLAINTEXT.name, e.listenerName.value)
assertEquals(SecurityProtocol.PLAINTEXT.name, e.listener)
)
assertFalse(ctx.currentDefaultLogConfig.get().originals().containsKey(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG))
}

View File

@ -22,7 +22,7 @@ import java.util
import java.util.{Arrays, Collections, Properties}
import kafka.utils.TestUtils.assertBadConfigContainingMessage
import kafka.utils.{CoreUtils, TestUtils}
import org.apache.kafka.common.Node
import org.apache.kafka.common.{Endpoint, Node}
import org.apache.kafka.common.config.{ConfigException, SaslConfigs, SecurityConfig, SslConfigs, TopicConfig}
import org.apache.kafka.common.metrics.Sensor
import org.apache.kafka.common.network.ListenerName
@ -35,7 +35,6 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig}
import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.EndPoint
import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig}
import org.apache.kafka.server.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
@ -343,7 +342,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
assertEquals(
Seq(new EndPoint("lb1.example.com", 9000, ListenerName.normalised("CONTROLLER"), SecurityProtocol.PLAINTEXT)),
Seq(new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "lb1.example.com", 9000)),
config.effectiveAdvertisedControllerListeners
)
}
@ -359,7 +358,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
assertEquals(
Seq(new EndPoint("localhost", 9093, ListenerName.normalised("CONTROLLER"), SecurityProtocol.PLAINTEXT)),
Seq(new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "localhost", 9093)),
config.effectiveAdvertisedControllerListeners
)
}
@ -377,8 +376,8 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
assertEquals(
Seq(
new EndPoint("lb1.example.com", 9000, ListenerName.normalised("CONTROLLER"), SecurityProtocol.PLAINTEXT),
new EndPoint("localhost", 9094, ListenerName.normalised("CONTROLLER_NEW"), SecurityProtocol.PLAINTEXT)
new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "lb1.example.com", 9000),
new Endpoint("CONTROLLER_NEW", SecurityProtocol.PLAINTEXT, "localhost", 9094)
),
config.effectiveAdvertisedControllerListeners
)
@ -507,9 +506,9 @@ class KafkaConfigTest {
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "REPLICATION")
val config = KafkaConfig.fromProps(props)
val expectedListeners = Seq(
new EndPoint("localhost", 9091, new ListenerName("CLIENT"), SecurityProtocol.SSL),
new EndPoint("localhost", 9092, new ListenerName("REPLICATION"), SecurityProtocol.SSL),
new EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT))
new Endpoint("CLIENT", SecurityProtocol.SSL, "localhost", 9091),
new Endpoint("REPLICATION", SecurityProtocol.SSL, "localhost", 9092),
new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093))
assertEquals(expectedListeners, config.listeners)
assertEquals(expectedListeners, config.effectiveAdvertisedBrokerListeners)
val expectedSecurityProtocolMap = Map(
@ -536,14 +535,14 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props)
val expectedListeners = Seq(
new EndPoint("localhost", 9091, new ListenerName("EXTERNAL"), SecurityProtocol.SSL),
new EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)
new Endpoint("EXTERNAL", SecurityProtocol.SSL, "localhost", 9091),
new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093)
)
assertEquals(expectedListeners, config.listeners)
val expectedAdvertisedListeners = Seq(
new EndPoint("lb1.example.com", 9000, new ListenerName("EXTERNAL"), SecurityProtocol.SSL),
new EndPoint("host1", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)
new Endpoint("EXTERNAL", SecurityProtocol.SSL, "lb1.example.com", 9000),
new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "host1", 9093)
)
assertEquals(expectedAdvertisedListeners, config.effectiveAdvertisedBrokerListeners)
@ -593,8 +592,8 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092")
props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,SSL:SSL,CONTROLLER:PLAINTEXT")
val config = KafkaConfig.fromProps(props)
assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listenerName.value == "SSL").map(JTestUtils.endpointToString))
assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listenerName.value == "PLAINTEXT").map(JTestUtils.endpointToString))
assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listener == "SSL").map(JTestUtils.endpointToString))
assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listener == "PLAINTEXT").map(JTestUtils.endpointToString))
}
private def listenerListToEndPoints(listenerList: String,

View File

@ -66,7 +66,7 @@ class RegistrationTestContext(
val controllerEpoch = new AtomicInteger(123)
config.effectiveAdvertisedBrokerListeners.foreach { ep =>
advertisedListeners.add(new Listener().setHost(ep.host).
setName(ep.listenerName.value()).
setName(ep.listener).
setPort(ep.port.shortValue()).
setSecurityProtocol(ep.securityProtocol.id))
}

View File

@ -181,7 +181,7 @@ object TestUtils extends Logging {
listenerName: ListenerName
): String = {
brokers.map { s =>
val listener = s.config.effectiveAdvertisedBrokerListeners.find(_.listenerName == listenerName).getOrElse(
val listener = s.config.effectiveAdvertisedBrokerListeners.find(_.listener == listenerName.value).getOrElse(
sys.error(s"Could not find listener with name ${listenerName.value}"))
formatAddress(listener.host, s.boundPort(listenerName))
}.mkString(",")

View File

@ -85,7 +85,7 @@ public class BrokerRegistration {
public Builder setListeners(List<Endpoint> listeners) {
Map<String, Endpoint> listenersMap = new HashMap<>();
for (Endpoint endpoint : listeners) {
listenersMap.put(endpoint.listenerName().get(), endpoint);
listenersMap.put(endpoint.listener(), endpoint);
}
this.listeners = listenersMap;
return this;
@ -170,7 +170,7 @@ public class BrokerRegistration {
this.incarnationId = incarnationId;
Map<String, Endpoint> newListeners = new HashMap<>(listeners.size());
for (Entry<String, Endpoint> entry : listeners.entrySet()) {
if (entry.getValue().listenerName().isEmpty()) {
if (entry.getValue().listener().isEmpty()) {
throw new IllegalArgumentException("Broker listeners must be named.");
}
newListeners.put(entry.getKey(), entry.getValue());

View File

@ -188,14 +188,14 @@ public final class ListenerInfo {
) {
LinkedHashMap<String, Endpoint> listeners = new LinkedHashMap<>();
for (Endpoint listener : rawListeners) {
String name = listener.listenerName().get();
String name = listener.listener();
if (Optional.of(name).equals(firstListenerName)) {
listeners.put(name, listener);
break;
}
}
for (Endpoint listener : rawListeners) {
String name = listener.listenerName().get();
String name = listener.listener();
if (!Optional.of(name).equals(firstListenerName)) {
listeners.put(name, listener);
}
@ -236,11 +236,11 @@ public final class ListenerInfo {
if (entry.getValue().host() == null || entry.getValue().host().trim().isEmpty()) {
String newHost = InetAddress.getLocalHost().getCanonicalHostName();
Endpoint prevEndpoint = entry.getValue();
newListeners.put(entry.getKey(), new Endpoint(prevEndpoint.listenerName().get(),
newListeners.put(entry.getKey(), new Endpoint(prevEndpoint.listener(),
prevEndpoint.securityProtocol(),
newHost,
prevEndpoint.port()));
log.info("{}: resolved wildcard host to {}", entry.getValue().listenerName().get(),
log.info("{}: resolved wildcard host to {}", entry.getValue().listener(),
newHost);
} else {
newListeners.put(entry.getKey(), entry.getValue());
@ -268,9 +268,9 @@ public final class ListenerInfo {
Endpoint prevEndpoint = entry.getValue();
int newPort = getBoundPortCallback.apply(entry.getKey());
checkPortIsSerializable(newPort);
log.info("{}: resolved ephemeral port to {}", entry.getValue().listenerName().get(),
log.info("{}: resolved ephemeral port to {}", entry.getValue().listener(),
newPort);
newListeners.put(entry.getKey(), new Endpoint(prevEndpoint.listenerName().get(),
newListeners.put(entry.getKey(), new Endpoint(prevEndpoint.listener(),
prevEndpoint.securityProtocol(),
prevEndpoint.host(),
newPort));
@ -309,7 +309,7 @@ public final class ListenerInfo {
checkHostIsSerializable(endpoint.host());
collection.add(new ControllerRegistrationRequestData.Listener().
setHost(endpoint.host()).
setName(endpoint.listenerName().get()).
setName(endpoint.listener()).
setPort(endpoint.port()).
setSecurityProtocol(endpoint.securityProtocol().id));
});
@ -324,7 +324,7 @@ public final class ListenerInfo {
checkHostIsSerializable(endpoint.host());
collection.add(new RegisterControllerRecord.ControllerEndpoint().
setHost(endpoint.host()).
setName(endpoint.listenerName().get()).
setName(endpoint.listener()).
setPort(endpoint.port()).
setSecurityProtocol(endpoint.securityProtocol().id));
});
@ -339,7 +339,7 @@ public final class ListenerInfo {
checkHostIsSerializable(endpoint.host());
collection.add(new BrokerRegistrationRequestData.Listener().
setHost(endpoint.host()).
setName(endpoint.listenerName().get()).
setName(endpoint.listener()).
setPort(endpoint.port()).
setSecurityProtocol(endpoint.securityProtocol().id));
});
@ -354,7 +354,7 @@ public final class ListenerInfo {
checkHostIsSerializable(endpoint.host());
collection.add(new RegisterBrokerRecord.BrokerEndpoint().
setHost(endpoint.host()).
setName(endpoint.listenerName().get()).
setName(endpoint.listener()).
setPort(endpoint.port()).
setSecurityProtocol(endpoint.securityProtocol().id));
});

View File

@ -132,7 +132,7 @@ public class StandardAuthorizer implements ClusterMetadataAuthorizer, Monitorabl
Map<Endpoint, CompletableFuture<Void>> result = new HashMap<>();
for (Endpoint endpoint : serverInfo.endpoints()) {
if (serverInfo.earlyStartListeners().contains(
endpoint.listenerName().orElse(""))) {
endpoint.listener())) {
result.put(endpoint, CompletableFuture.completedFuture(null));
} else {
result.put(endpoint, initialLoadFuture);

View File

@ -27,6 +27,7 @@ import org.junit.jupiter.params.provider.ValueSource;
import java.util.ArrayList;
import java.util.List;
import java.util.Optional;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
@ -93,16 +94,16 @@ public class ListenerInfoTest {
endpoints.add(ALL.get((i + startIndex) % ALL.size()));
}
ListenerInfo listenerInfo = ListenerInfo.create(endpoints);
assertEquals(ALL.get(startIndex).listenerName().get(),
listenerInfo.firstListener().listenerName().get());
assertEquals(ALL.get(startIndex).listener(),
listenerInfo.firstListener().listener());
}
@ParameterizedTest
@ValueSource(ints = {0, 1, 2, 3})
public void testCreateWithExplicitFirstListener(int startIndex) {
ListenerInfo listenerInfo = ListenerInfo.create(ALL.get(startIndex).listenerName(), ALL);
assertEquals(ALL.get(startIndex).listenerName().get(),
listenerInfo.firstListener().listenerName().get());
ListenerInfo listenerInfo = ListenerInfo.create(Optional.of(ALL.get(startIndex).listener()), ALL);
assertEquals(ALL.get(startIndex).listener(),
listenerInfo.firstListener().listener());
}
@Test

View File

@ -125,8 +125,8 @@ public class StandardAuthorizerTest {
public Collection<String> earlyStartListeners() {
List<String> result = new ArrayList<>();
for (Endpoint endpoint : endpoints) {
if (endpoint.listenerName().get().equals("CONTROLLER")) {
result.add(endpoint.listenerName().get());
if (endpoint.listener().equals("CONTROLLER")) {
result.add(endpoint.listener());
}
}
return result;

View File

@ -113,7 +113,7 @@ public class EndpointReadyFutures {
List<String> notInInfo = new ArrayList<>();
for (Endpoint endpoint : effectiveStartFutures.keySet()) {
if (!info.endpoints().contains(endpoint)) {
notInInfo.add(endpoint.listenerName().orElse("[none]"));
notInInfo.add(endpoint.listener());
}
}
throw new RuntimeException("Found authorizer futures that weren't included " +
@ -146,7 +146,7 @@ public class EndpointReadyFutures {
final CompletableFuture<Void> future;
EndpointReadyFuture(Endpoint endpoint, Collection<String> stageNames) {
this.endpointName = endpoint.listenerName().orElse("UNNAMED");
this.endpointName = endpoint.listener();
this.incomplete = new TreeSet<>(stageNames);
this.future = new CompletableFuture<>();
}

View File

@ -56,7 +56,7 @@ public final class EndpointReadyFuturesTest {
Endpoint... endpoints
) {
for (Endpoint endpoint : endpoints) {
String name = endpoint.listenerName().get();
String name = endpoint.listener();
CompletableFuture<Void> future = readyFutures.futures().get(endpoint);
assertNotNull(future, "Unable to find future for " + name);
assertTrue(future.isDone(), "Future for " + name + " is not done.");

View File

@ -1,48 +0,0 @@
/*
* 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.network;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.security.auth.SecurityProtocol;
import java.util.Locale;
public record EndPoint(
String host,
int port,
ListenerName listenerName,
SecurityProtocol securityProtocol
) {
public static String parseListenerName(String connectionString) {
int firstColon = connectionString.indexOf(':');
if (firstColon < 0) {
throw new KafkaException("Unable to parse a listener name from " + connectionString);
}
return connectionString.substring(0, firstColon).toUpperCase(Locale.ROOT);
}
public static EndPoint fromPublic(org.apache.kafka.common.Endpoint endpoint) {
return new EndPoint(endpoint.host(), endpoint.port(),
new ListenerName(endpoint.listenerName().get()), endpoint.securityProtocol());
}
public org.apache.kafka.common.Endpoint toPublic() {
return new org.apache.kafka.common.Endpoint(listenerName.value(), securityProtocol, host, port);
}
}

View File

@ -18,13 +18,13 @@ package org.apache.kafka.common.test;
import org.apache.kafka.clients.admin.Admin;
import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.network.EndPoint;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -106,10 +106,10 @@ public class TestUtils {
/**
* Convert EndPoint to String
*/
public static String endpointToString(EndPoint endPoint) {
public static String endpointToString(Endpoint endPoint) {
String host = endPoint.host();
int port = endPoint.port();
ListenerName listenerName = endPoint.listenerName();
ListenerName listenerName = ListenerName.normalised(endPoint.listener());
String hostport = (host == null) ? (":" + port) : Utils.formatAddress(host, port);
return listenerName.value() + "://" + hostport;

View File

@ -395,10 +395,10 @@ public class MetadataQuorumCommand {
Map<String, Endpoint> listeners = new HashMap<>();
SocketServerConfigs.listenerListToEndPoints(
props.getOrDefault(SocketServerConfigs.LISTENERS_CONFIG, "").toString(),
__ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listenerName().get(), e));
__ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e));
SocketServerConfigs.listenerListToEndPoints(
props.getOrDefault(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "").toString(),
__ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listenerName().get(), e));
__ -> SecurityProtocol.PLAINTEXT).forEach(e -> listeners.put(e.listener(), e));
if (!props.containsKey(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG)) {
throw new TerseException(KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG +
" was not found. Is this a valid controller configuration file?");
@ -412,7 +412,7 @@ public class MetadataQuorumCommand {
throw new TerseException("Cannot find information about controller listener name: " +
listenerName);
}
results.add(new RaftVoterEndpoint(endpoint.listenerName().get(),
results.add(new RaftVoterEndpoint(endpoint.listener(),
endpoint.host() == null ? "localhost" : endpoint.host(),
endpoint.port()));
}
@ -443,7 +443,7 @@ public class MetadataQuorumCommand {
output.append(" and endpoints: ");
String prefix = "";
for (RaftVoterEndpoint endpoint : endpoints) {
output.append(prefix).append(endpoint.name()).append("://");
output.append(prefix).append(endpoint.listener()).append("://");
if (endpoint.host().contains(":")) {
output.append("[");
}