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(); new AddRaftVoterRequestData.ListenerCollection();
endpoints.forEach(endpoint -> endpoints.forEach(endpoint ->
listeners.add(new AddRaftVoterRequestData.Listener(). listeners.add(new AddRaftVoterRequestData.Listener().
setName(endpoint.name()). setName(endpoint.listener()).
setHost(endpoint.host()). setHost(endpoint.host()).
setPort(endpoint.port()))); setPort(endpoint.port())));
return new AddRaftVoterRequest.Builder( return new AddRaftVoterRequest.Builder(

View File

@ -26,7 +26,7 @@ import java.util.Objects;
*/ */
@InterfaceStability.Stable @InterfaceStability.Stable
public class RaftVoterEndpoint { public class RaftVoterEndpoint {
private final String name; private final String listener;
private final String host; private final String host;
private final int port; private final int port;
@ -49,22 +49,33 @@ public class RaftVoterEndpoint {
/** /**
* Create an endpoint for a metadata quorum voter. * 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 host The DNS hostname for this endpoint.
* @param port The network port for this endpoint. * @param port The network port for this endpoint.
*/ */
public RaftVoterEndpoint( public RaftVoterEndpoint(
String name, String listener,
String host, String host,
int port int port
) { ) {
this.name = requireNonNullAllCapsNonEmpty(name); this.listener = requireNonNullAllCapsNonEmpty(listener);
this.host = Objects.requireNonNull(host); this.host = Objects.requireNonNull(host);
this.port = port; 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() { public String name() {
return name; return listener;
} }
public String host() { public String host() {
@ -79,20 +90,20 @@ public class RaftVoterEndpoint {
public boolean equals(Object o) { public boolean equals(Object o) {
if (o == null || (!o.getClass().equals(getClass()))) return false; if (o == null || (!o.getClass().equals(getClass()))) return false;
RaftVoterEndpoint other = (RaftVoterEndpoint) o; RaftVoterEndpoint other = (RaftVoterEndpoint) o;
return name.equals(other.name) && return listener.equals(other.listener) &&
host.equals(other.host) && host.equals(other.host) &&
port == other.port; port == other.port;
} }
@Override @Override
public int hashCode() { public int hashCode() {
return Objects.hash(name, host, port); return Objects.hash(listener, host, port);
} }
@Override @Override
public String toString() { public String toString() {
// enclose IPv6 hosts in square brackets for readability // enclose IPv6 hosts in square brackets for readability
String hostString = host.contains(":") ? "[" + host + "]" : host; 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; package org.apache.kafka.common;
import org.apache.kafka.common.annotation.InterfaceStability;
import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.security.auth.SecurityProtocol;
import java.util.Objects; import java.util.Objects;
@ -26,27 +25,35 @@ import java.util.Optional;
* Represents a broker endpoint. * Represents a broker endpoint.
*/ */
@InterfaceStability.Evolving
public class Endpoint { public class Endpoint {
private final String listenerName; private final String listener;
private final SecurityProtocol securityProtocol; private final SecurityProtocol securityProtocol;
private final String host; private final String host;
private final int port; private final int port;
public Endpoint(String listenerName, SecurityProtocol securityProtocol, String host, int port) { public Endpoint(String listener, SecurityProtocol securityProtocol, String host, int port) {
this.listenerName = listenerName; this.listener = listener;
this.securityProtocol = securityProtocol; this.securityProtocol = securityProtocol;
this.host = host; this.host = host;
this.port = port; 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 * 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. * 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() { public Optional<String> listenerName() {
return Optional.ofNullable(listenerName); return Optional.ofNullable(listener);
} }
/** /**
@ -80,7 +87,7 @@ public class Endpoint {
} }
Endpoint that = (Endpoint) o; 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.securityProtocol, that.securityProtocol) &&
Objects.equals(this.host, that.host) && Objects.equals(this.host, that.host) &&
this.port == that.port; this.port == that.port;
@ -89,13 +96,13 @@ public class Endpoint {
@Override @Override
public int hashCode() { public int hashCode() {
return Objects.hash(listenerName, securityProtocol, host, port); return Objects.hash(listener, securityProtocol, host, port);
} }
@Override @Override
public String toString() { public String toString() {
return "Endpoint(" + return "Endpoint(" +
"listenerName='" + listenerName + '\'' + "listenerName='" + listener + '\'' +
", securityProtocol=" + securityProtocol + ", securityProtocol=" + securityProtocol +
", host='" + host + '\'' + ", host='" + host + '\'' +
", port=" + port + ", port=" + port +

View File

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

View File

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

View File

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

View File

@ -27,7 +27,7 @@ import kafka.raft.KafkaRaftManager
import kafka.server.DynamicBrokerConfig._ import kafka.server.DynamicBrokerConfig._
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import org.apache.kafka.common.Reconfigurable 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.internals.BrokerSecurityConfigs
import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SslConfigs} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef, ConfigException, ConfigResource, SaslConfigs, SslConfigs}
import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType} import org.apache.kafka.common.metadata.{ConfigRecord, MetadataRecordType}
@ -961,9 +961,9 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
def validateReconfiguration(newConfig: KafkaConfig): Unit = { def validateReconfiguration(newConfig: KafkaConfig): Unit = {
val oldConfig = server.config val oldConfig = server.config
val newListeners = newConfig.listeners.map(_.listenerName).toSet val newListeners = newConfig.listeners.map(l => ListenerName.normalised(l.listener)).toSet
val oldAdvertisedListeners = oldConfig.effectiveAdvertisedBrokerListeners.map(_.listenerName).toSet val oldAdvertisedListeners = oldConfig.effectiveAdvertisedBrokerListeners.map(l => ListenerName.normalised(l.listener)).toSet
val oldListeners = oldConfig.listeners.map(_.listenerName).toSet val oldListeners = oldConfig.listeners.map(l => ListenerName.normalised(l.listener)).toSet
if (!oldAdvertisedListeners.subsetOf(newListeners)) if (!oldAdvertisedListeners.subsetOf(newListeners))
throw new ConfigException(s"Advertised listeners '$oldAdvertisedListeners' must be a subset of listeners '$newListeners'") throw new ConfigException(s"Advertised listeners '$oldAdvertisedListeners' must be a subset of listeners '$newListeners'")
if (!newListeners.subsetOf(newConfig.effectiveListenerSecurityProtocolMap.keySet)) if (!newListeners.subsetOf(newConfig.effectiveListenerSecurityProtocolMap.keySet))
@ -988,8 +988,8 @@ class DynamicListenerConfig(server: KafkaBroker) extends BrokerReconfigurable wi
val newListenerMap = listenersToMap(newListeners) val newListenerMap = listenersToMap(newListeners)
val oldListeners = oldConfig.listeners val oldListeners = oldConfig.listeners
val oldListenerMap = listenersToMap(oldListeners) val oldListenerMap = listenersToMap(oldListeners)
val listenersRemoved = oldListeners.filterNot(e => newListenerMap.contains(e.listenerName)) val listenersRemoved = oldListeners.filterNot(e => newListenerMap.contains(ListenerName.normalised(e.listener)))
val listenersAdded = newListeners.filterNot(e => oldListenerMap.contains(e.listenerName)) val listenersAdded = newListeners.filterNot(e => oldListenerMap.contains(ListenerName.normalised(e.listener)))
if (listenersRemoved.nonEmpty || listenersAdded.nonEmpty) { if (listenersRemoved.nonEmpty || listenersAdded.nonEmpty) {
LoginManager.closeAll() // Clear SASL login cache to force re-login LoginManager.closeAll() // Clear SASL login cache to force re-login
if (listenersRemoved.nonEmpty) server.socketServer.removeListeners(listenersRemoved) 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] = private def listenersToMap(listeners: Seq[Endpoint]): Map[ListenerName, Endpoint] =
listeners.map(e => (e.listenerName, e)).toMap 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 java.util.Properties
import kafka.utils.{CoreUtils, Logging} import kafka.utils.{CoreUtils, Logging}
import kafka.utils.Implicits._ 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, ConfigException, ConfigResource, TopicConfig}
import org.apache.kafka.common.config.ConfigDef.ConfigKey import org.apache.kafka.common.config.ConfigDef.ConfigKey
import org.apache.kafka.common.config.internals.BrokerSecurityConfigs 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.KafkaPrincipalSerde
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils 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.Group.GroupType
import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig
import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig}
@ -145,6 +144,14 @@ object KafkaConfig {
} }
output 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 earlyStartListeners: Set[ListenerName] = {
val listenersSet = listeners.map(_.listenerName).toSet val listenersSet = listeners.map(l => ListenerName.normalised(l.listener)).toSet
val controllerListenersSet = controllerListeners.map(_.listenerName).toSet val controllerListenersSet = controllerListeners.map(l => ListenerName.normalised(l.listener)).toSet
Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match { Option(getString(ServerConfigs.EARLY_START_LISTENERS_CONFIG)) match {
case None => controllerListenersSet case None => controllerListenersSet
case Some(str) => 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) CoreUtils.listenerListToEndPoints(getString(SocketServerConfigs.LISTENERS_CONFIG), effectiveListenerSecurityProtocolMap)
def controllerListenerNames: Seq[String] = { def controllerListenerNames: Seq[String] = {
@ -471,23 +478,23 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
} }
} }
def controllerListeners: Seq[EndPoint] = def controllerListeners: Seq[Endpoint] =
listeners.filter(l => controllerListenerNames.contains(l.listenerName.value())) listeners.filter(l => controllerListenerNames.contains(l.listener))
def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG) def saslMechanismControllerProtocol: String = getString(KRaftConfigs.SASL_MECHANISM_CONTROLLER_PROTOCOL_CONFIG)
def dataPlaneListeners: Seq[EndPoint] = { def dataPlaneListeners: Seq[Endpoint] = {
listeners.filterNot { listener => listeners.filterNot { listener =>
val name = listener.listenerName.value() val name = listener.listener
controllerListenerNames.contains(name) controllerListenerNames.contains(name)
} }
} }
def effectiveAdvertisedControllerListeners: Seq[EndPoint] = { def effectiveAdvertisedControllerListeners: Seq[Endpoint] = {
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
val controllerAdvertisedListeners = if (advertisedListenersProp != null) { val controllerAdvertisedListeners = if (advertisedListenersProp != null) {
CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false) CoreUtils.listenerListToEndPoints(advertisedListenersProp, effectiveListenerSecurityProtocolMap, requireDistinctPorts=false)
.filter(l => controllerListenerNames.contains(l.listenerName.value())) .filter(l => controllerListenerNames.contains(l.listener))
} else { } else {
Seq.empty Seq.empty
} }
@ -495,16 +502,16 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
controllerListenerNames.flatMap { name => controllerListenerNames.flatMap { name =>
controllerAdvertisedListeners controllerAdvertisedListeners
.find(endpoint => endpoint.listenerName.equals(ListenerName.normalised(name))) .find(endpoint => ListenerName.normalised(endpoint.listener).equals(ListenerName.normalised(name)))
.orElse( .orElse(
// If users don't define advertised.listeners, the advertised controller listeners inherit from listeners configuration // If users don't define advertised.listeners, the advertised controller listeners inherit from listeners configuration
// which match listener names in controller.listener.names. // 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. // 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. // The null or "" host does a reverse lookup in ListenerInfo#withWildcardHostnamesResolved.
controllerListenersValue 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") { .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 { } else {
endpoint 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 // Use advertised listeners if defined, fallback to listeners otherwise
val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG) val advertisedListenersProp = getString(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG)
val advertisedListeners = if (advertisedListenersProp != null) { val advertisedListeners = if (advertisedListenersProp != null) {
@ -521,7 +528,7 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
listeners listeners
} }
// Only expose broker 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) = { 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) // check controller listener names (they won't appear in listeners when process.roles=broker)
// as well as listeners for occurrences of SSL or SASL_* // as well as listeners for occurrences of SSL or SASL_*
if (controllerListenerNames.exists(isSslOrSasl) || 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_* mapValue // don't add default mappings since we found something that is SSL or SASL_*
} else { } else {
// add the PLAINTEXT mappings for all controller listener names that are not explicitly PLAINTEXT // 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" + 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") " 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 // validate KRaft-related configs
val voterIds = QuorumConfig.parseVoterIds(quorumConfig.voters) 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") s"${KRaftConfigs.CONTROLLER_LISTENER_NAMES_CONFIG} must contain at least one value appearing in the '${SocketServerConfigs.LISTENERS_CONFIG}' configuration when running the KRaft controller role")
} }
def validateControllerListenerNamesMustAppearInListenersForKRaftController(): Unit = { def validateControllerListenerNamesMustAppearInListenersForKRaftController(): Unit = {
val listenerNameValues = listeners.map(_.listenerName.value).toSet val listenerNameValues = listeners.map(_.listener).toSet
require(controllerListenerNames.forall(cln => listenerNameValues.contains(cln)), 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") 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() validateControllerListenerNamesMustAppearInListenersForKRaftController()
} }
val listenerNames = listeners.map(_.listenerName).toSet val listenerNames = listeners.map(l => ListenerName.normalised(l.listener)).toSet
if (processRoles.contains(ProcessRole.BrokerRole)) { if (processRoles.contains(ProcessRole.BrokerRole)) {
validateAdvertisedBrokerListenersNonEmptyForBroker() validateAdvertisedBrokerListenersNonEmptyForBroker()
require(advertisedBrokerListenerNames.contains(interBrokerListenerName), 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.Metrics
import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing
import org.apache.kafka.common.metrics.stats.{Meter, Percentile, Percentiles} 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.protocol.{ObjectSerializationCache, Writable}
import org.apache.kafka.common.security.scram.internals.ScramMechanism import org.apache.kafka.common.security.scram.internals.ScramMechanism
import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache import org.apache.kafka.common.security.token.delegation.internals.DelegationTokenCache
@ -88,7 +89,7 @@ class TestRaftServer(
val endpoints = Endpoints.fromInetSocketAddresses( val endpoints = Endpoints.fromInetSocketAddresses(
config.effectiveAdvertisedControllerListeners config.effectiveAdvertisedControllerListeners
.map { endpoint => .map { endpoint =>
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port)) (ListenerName.normalised(endpoint.listener), InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
} }
.toMap .toMap
.asJava .asJava

View File

@ -25,8 +25,8 @@ import com.typesafe.scalalogging.Logger
import javax.management.ObjectName import javax.management.ObjectName
import scala.collection._ import scala.collection._
import scala.collection.Seq import scala.collection.Seq
import org.apache.kafka.network.EndPoint
import org.apache.commons.validator.routines.InetAddressValidator 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.network.ListenerName
import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.utils.Utils 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 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) 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 val distinctPorts = endpoints.map(_.port).distinct
require(distinctPorts.size == endpoints.map(_.port).size, s"Each listener must have a different port, listeners: $listeners") 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 = def validateOneIsIpv4AndOtherIpv6(first: String, second: String): Boolean =
(inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) || (inetAddressValidator.isValidInet4Address(first) && inetAddressValidator.isValidInet6Address(second)) ||
(inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second)) (inetAddressValidator.isValidInet6Address(first) && inetAddressValidator.isValidInet4Address(second))
def validate(endPoints: Seq[EndPoint]): Unit = { def validate(endPoints: Seq[Endpoint]): Unit = {
val distinctListenerNames = endPoints.map(_.listenerName).distinct val distinctListenerNames = endPoints.map(_.listener).distinct
require(distinctListenerNames.size == endPoints.size, s"Each listener must have a different name, listeners: $listeners") require(distinctListenerNames.size == endPoints.size, s"Each listener must have a different name, listeners: $listeners")
val (duplicatePorts, _) = endPoints.filter { val (duplicatePorts, _) = endPoints.filter {
@ -186,8 +186,7 @@ object CoreUtils {
} }
val endPoints = try { val endPoints = try {
SocketServerConfigs.listenerListToEndPoints(listeners, securityProtocolMap.asJava). SocketServerConfigs.listenerListToEndPoints(listeners, securityProtocolMap.asJava).asScala
asScala.map(EndPoint.fromPublic)
} catch { } catch {
case e: Exception => case e: Exception =>
throw new IllegalArgumentException(s"Error creating broker listeners from '$listeners': ${e.getMessage}", e) 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 controllerListenerName = ListenerName.forSecurityProtocol(extraControllerSecurityProtocol)
val config = controllerServers.map { s => val config = controllerServers.map { s =>
val listener = s.config.effectiveAdvertisedControllerListeners val listener = s.config.effectiveAdvertisedControllerListeners
.find(_.listenerName == controllerListenerName) .find(_.listener == controllerListenerName.value)
.getOrElse(throw new IllegalArgumentException(s"Could not find listener with name $controllerListenerName")) .getOrElse(throw new IllegalArgumentException(s"Could not find listener with name $controllerListenerName"))
Utils.formatAddress(listener.host, s.socketServer.boundPort(controllerListenerName)) Utils.formatAddress(listener.host, s.socketServer.boundPort(controllerListenerName))
}.mkString(",") }.mkString(",")

View File

@ -113,7 +113,7 @@ class KRaftClusterTest {
cluster.format() cluster.format()
cluster.startup() cluster.startup()
val controller = cluster.controllers().values().iterator().asScala.filter(_.controller.isActive).next() 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 // shutdown active controller
controller.shutdown() controller.shutdown()

View File

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

View File

@ -22,6 +22,7 @@ import com.yammer.metrics.core.{Gauge, Meter}
import kafka.server._ import kafka.server._
import kafka.utils.Implicits._ import kafka.utils.Implicits._
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.common.Endpoint
import org.apache.kafka.common.memory.MemoryPool import org.apache.kafka.common.memory.MemoryPool
import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiMessageType.ListenerType
import org.apache.kafka.common.message.{ProduceRequestData, SaslAuthenticateRequestData, SaslHandshakeRequestData, VoteRequestData} 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.common.utils._
import org.apache.kafka.network.RequestConvertToJson import org.apache.kafka.network.RequestConvertToJson
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.EndPoint
import org.apache.kafka.security.CredentialProvider import org.apache.kafka.security.CredentialProvider
import org.apache.kafka.server.{ApiVersionManager, SimpleApiVersionManager} import org.apache.kafka.server.{ApiVersionManager, SimpleApiVersionManager}
import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion} import org.apache.kafka.server.common.{FinalizedFeatures, MetadataVersion}
@ -91,10 +91,10 @@ class SocketServerTest {
private val kafkaLogger = LogManager.getLogger("kafka") private val kafkaLogger = LogManager.getLogger("kafka")
private var logLevelToRestore: Level = _ private var logLevelToRestore: Level = _
def endpoint: EndPoint = { def endpoint: Endpoint = {
KafkaConfig.fromProps(props, doLog = false).dataPlaneListeners.head KafkaConfig.fromProps(props, doLog = false).dataPlaneListeners.head
} }
def listener: String = endpoint.listenerName.value def listener: String = endpoint.listener
val uncaughtExceptions = new AtomicInteger(0) val uncaughtExceptions = new AtomicInteger(0)
@BeforeEach @BeforeEach
@ -840,7 +840,7 @@ class SocketServerTest {
// same as SocketServer.createAcceptor, // same as SocketServer.createAcceptor,
// except the Acceptor overriding a method to inject the exception // 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) { 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 = { override protected def configureAcceptedSocketChannel(socketChannel: SocketChannel): Unit = {
@ -1858,7 +1858,7 @@ class SocketServerTest {
val failedFuture = new CompletableFuture[Void]() val failedFuture = new CompletableFuture[Void]()
failedFuture.completeExceptionally(new RuntimeException("authorizer startup failed")) failedFuture.completeExceptionally(new RuntimeException("authorizer startup failed"))
assertThrows(classOf[ExecutionException], () => { assertThrows(classOf[ExecutionException], () => {
newServer.enableRequestProcessing(Map(endpoint.toPublic -> failedFuture)).get() newServer.enableRequestProcessing(Map(endpoint -> failedFuture)).get()
}) })
} finally { } finally {
shutdownServerAndMetrics(newServer) shutdownServerAndMetrics(newServer)
@ -1891,7 +1891,7 @@ class SocketServerTest {
val authorizerFuture = new CompletableFuture[Void]() val authorizerFuture = new CompletableFuture[Void]()
val enableFuture = newServer.enableRequestProcessing( val enableFuture = newServer.enableRequestProcessing(
newServer.dataPlaneAcceptors.keys().asScala. newServer.dataPlaneAcceptors.keys().asScala.
map(_.toPublic).map(k => k -> authorizerFuture).toMap) map(k => k -> authorizerFuture).toMap)
assertFalse(authorizerFuture.isDone) assertFalse(authorizerFuture.isDone)
assertFalse(enableFuture.isDone) assertFalse(enableFuture.isDone)
newServer.dataPlaneAcceptors.values().forEach(a => assertNull(a.serverChannel)) newServer.dataPlaneAcceptors.values().forEach(a => assertNull(a.serverChannel))
@ -1992,7 +1992,7 @@ class SocketServerTest {
} }
class TestableAcceptor(socketServer: SocketServer, class TestableAcceptor(socketServer: SocketServer,
endPoint: EndPoint, endPoint: Endpoint,
cfg: KafkaConfig, cfg: KafkaConfig,
nodeId: Int, nodeId: Int,
connectionQuotas: ConnectionQuotas, connectionQuotas: ConnectionQuotas,
@ -2098,7 +2098,7 @@ class SocketServerTest {
connectionDisconnectListeners = connectionDisconnectListeners 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) 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.TopicPartition
import org.apache.kafka.common.Uuid import org.apache.kafka.common.Uuid
import org.apache.kafka.common.metrics.Metrics 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.common.utils.Time
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig} import org.apache.kafka.raft.{Endpoints, MetadataLogConfig, QuorumConfig}
@ -88,7 +89,7 @@ class RaftManagerTest {
val endpoints = Endpoints.fromInetSocketAddresses( val endpoints = Endpoints.fromInetSocketAddresses(
config.effectiveAdvertisedControllerListeners config.effectiveAdvertisedControllerListeners
.map { endpoint => .map { endpoint =>
(endpoint.listenerName, InetSocketAddress.createUnresolved(endpoint.host, endpoint.port)) (ListenerName.normalised(endpoint.listener), InetSocketAddress.createUnresolved(endpoint.host, endpoint.port))
} }
.toMap .toMap
.asJava .asJava

View File

@ -76,7 +76,7 @@ class ControllerRegistrationManagerTest {
"controller-registration-manager-test-", "controller-registration-manager-test-",
createSupportedFeatures(MetadataVersion.IBP_3_7_IV0), createSupportedFeatures(MetadataVersion.IBP_3_7_IV0),
RecordTestUtils.createTestControllerRegistration(1, false).incarnationId(), 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)) 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") props.put(SocketServerConfigs.ADVERTISED_LISTENERS_CONFIG, "SASL_PLAINTEXT://localhost:8181")
ctx.config.dynamicConfig.updateDefaultConfig(props) ctx.config.dynamicConfig.updateDefaultConfig(props)
ctx.config.effectiveAdvertisedBrokerListeners.foreach(e => 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)) 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 java.util.{Arrays, Collections, Properties}
import kafka.utils.TestUtils.assertBadConfigContainingMessage import kafka.utils.TestUtils.assertBadConfigContainingMessage
import kafka.utils.{CoreUtils, TestUtils} 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.config.{ConfigException, SaslConfigs, SecurityConfig, SslConfigs, TopicConfig}
import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.metrics.Sensor
import org.apache.kafka.common.network.ListenerName 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.group.modern.share.ShareGroupConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig} import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig}
import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.network.SocketServerConfigs
import org.apache.kafka.network.EndPoint
import org.apache.kafka.raft.{MetadataLogConfig, QuorumConfig} 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.config.{DelegationTokenManagerConfigs, KRaftConfigs, QuotaConfig, ReplicationConfigs, ServerConfigs, ServerLogConfigs, ServerTopicConfigSynonyms}
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
@ -343,7 +342,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals( 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 config.effectiveAdvertisedControllerListeners
) )
} }
@ -359,7 +358,7 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals( assertEquals(
Seq(new EndPoint("localhost", 9093, ListenerName.normalised("CONTROLLER"), SecurityProtocol.PLAINTEXT)), Seq(new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "localhost", 9093)),
config.effectiveAdvertisedControllerListeners config.effectiveAdvertisedControllerListeners
) )
} }
@ -377,8 +376,8 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals( assertEquals(
Seq( Seq(
new EndPoint("lb1.example.com", 9000, ListenerName.normalised("CONTROLLER"), SecurityProtocol.PLAINTEXT), new Endpoint("CONTROLLER", SecurityProtocol.PLAINTEXT, "lb1.example.com", 9000),
new EndPoint("localhost", 9094, ListenerName.normalised("CONTROLLER_NEW"), SecurityProtocol.PLAINTEXT) new Endpoint("CONTROLLER_NEW", SecurityProtocol.PLAINTEXT, "localhost", 9094)
), ),
config.effectiveAdvertisedControllerListeners config.effectiveAdvertisedControllerListeners
) )
@ -507,9 +506,9 @@ class KafkaConfigTest {
props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "REPLICATION") props.setProperty(ReplicationConfigs.INTER_BROKER_LISTENER_NAME_CONFIG, "REPLICATION")
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
val expectedListeners = Seq( val expectedListeners = Seq(
new EndPoint("localhost", 9091, new ListenerName("CLIENT"), SecurityProtocol.SSL), new Endpoint("CLIENT", SecurityProtocol.SSL, "localhost", 9091),
new EndPoint("localhost", 9092, new ListenerName("REPLICATION"), SecurityProtocol.SSL), new Endpoint("REPLICATION", SecurityProtocol.SSL, "localhost", 9092),
new EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT)) new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093))
assertEquals(expectedListeners, config.listeners) assertEquals(expectedListeners, config.listeners)
assertEquals(expectedListeners, config.effectiveAdvertisedBrokerListeners) assertEquals(expectedListeners, config.effectiveAdvertisedBrokerListeners)
val expectedSecurityProtocolMap = Map( val expectedSecurityProtocolMap = Map(
@ -536,14 +535,14 @@ class KafkaConfigTest {
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
val expectedListeners = Seq( val expectedListeners = Seq(
new EndPoint("localhost", 9091, new ListenerName("EXTERNAL"), SecurityProtocol.SSL), new Endpoint("EXTERNAL", SecurityProtocol.SSL, "localhost", 9091),
new EndPoint("localhost", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT) new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "localhost", 9093)
) )
assertEquals(expectedListeners, config.listeners) assertEquals(expectedListeners, config.listeners)
val expectedAdvertisedListeners = Seq( val expectedAdvertisedListeners = Seq(
new EndPoint("lb1.example.com", 9000, new ListenerName("EXTERNAL"), SecurityProtocol.SSL), new Endpoint("EXTERNAL", SecurityProtocol.SSL, "lb1.example.com", 9000),
new EndPoint("host1", 9093, new ListenerName("INTERNAL"), SecurityProtocol.PLAINTEXT) new Endpoint("INTERNAL", SecurityProtocol.PLAINTEXT, "host1", 9093)
) )
assertEquals(expectedAdvertisedListeners, config.effectiveAdvertisedBrokerListeners) assertEquals(expectedAdvertisedListeners, config.effectiveAdvertisedBrokerListeners)
@ -593,8 +592,8 @@ class KafkaConfigTest {
props.setProperty(SocketServerConfigs.LISTENERS_CONFIG, "plaintext://localhost:9091,SsL://localhost:9092") 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") props.setProperty(SocketServerConfigs.LISTENER_SECURITY_PROTOCOL_MAP_CONFIG, "PLAINTEXT:PLAINTEXT,SSL:SSL,CONTROLLER:PLAINTEXT")
val config = KafkaConfig.fromProps(props) val config = KafkaConfig.fromProps(props)
assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listenerName.value == "SSL").map(JTestUtils.endpointToString)) assertEquals(Some("SSL://localhost:9092"), config.listeners.find(_.listener == "SSL").map(JTestUtils.endpointToString))
assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listenerName.value == "PLAINTEXT").map(JTestUtils.endpointToString)) assertEquals(Some("PLAINTEXT://localhost:9091"), config.listeners.find(_.listener == "PLAINTEXT").map(JTestUtils.endpointToString))
} }
private def listenerListToEndPoints(listenerList: String, private def listenerListToEndPoints(listenerList: String,

View File

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

View File

@ -181,7 +181,7 @@ object TestUtils extends Logging {
listenerName: ListenerName listenerName: ListenerName
): String = { ): String = {
brokers.map { s => 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}")) sys.error(s"Could not find listener with name ${listenerName.value}"))
formatAddress(listener.host, s.boundPort(listenerName)) formatAddress(listener.host, s.boundPort(listenerName))
}.mkString(",") }.mkString(",")

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -56,7 +56,7 @@ public final class EndpointReadyFuturesTest {
Endpoint... endpoints Endpoint... endpoints
) { ) {
for (Endpoint endpoint : endpoints) { for (Endpoint endpoint : endpoints) {
String name = endpoint.listenerName().get(); String name = endpoint.listener();
CompletableFuture<Void> future = readyFutures.futures().get(endpoint); CompletableFuture<Void> future = readyFutures.futures().get(endpoint);
assertNotNull(future, "Unable to find future for " + name); assertNotNull(future, "Unable to find future for " + name);
assertTrue(future.isDone(), "Future for " + name + " is not done."); 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.Admin;
import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.admin.TopicDescription;
import org.apache.kafka.common.Endpoint;
import org.apache.kafka.common.Node; import org.apache.kafka.common.Node;
import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.ListenerName;
import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Utils; import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.network.EndPoint;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -106,10 +106,10 @@ public class TestUtils {
/** /**
* Convert EndPoint to String * Convert EndPoint to String
*/ */
public static String endpointToString(EndPoint endPoint) { public static String endpointToString(Endpoint endPoint) {
String host = endPoint.host(); String host = endPoint.host();
int port = endPoint.port(); int port = endPoint.port();
ListenerName listenerName = endPoint.listenerName(); ListenerName listenerName = ListenerName.normalised(endPoint.listener());
String hostport = (host == null) ? (":" + port) : Utils.formatAddress(host, port); String hostport = (host == null) ? (":" + port) : Utils.formatAddress(host, port);
return listenerName.value() + "://" + hostport; return listenerName.value() + "://" + hostport;

View File

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