@ -21,6 +21,7 @@ import java.io.IOException
import java.net._
import java.net._
import java.nio.channels._
import java.nio.channels._
import java.nio.channels. { Selector => NSelector }
import java.nio.channels. { Selector => NSelector }
import java.util
import java.util.concurrent._
import java.util.concurrent._
import java.util.concurrent.atomic._
import java.util.concurrent.atomic._
import java.util.function.Supplier
import java.util.function.Supplier
@ -32,15 +33,16 @@ import kafka.network.RequestChannel.{CloseConnectionResponse, EndThrottlingRespo
import kafka.network.Processor._
import kafka.network.Processor._
import kafka.network.SocketServer._
import kafka.network.SocketServer._
import kafka.security.CredentialProvider
import kafka.security.CredentialProvider
import kafka.server.KafkaConfig
import kafka.server. { BrokerReconfigurable , KafkaConfig }
import kafka.utils._
import kafka.utils._
import org.apache.kafka.common.config.ConfigException
import org.apache.kafka.common. { KafkaException , Reconfigurable }
import org.apache.kafka.common. { KafkaException , Reconfigurable }
import org.apache.kafka.common.memory. { MemoryPool , SimpleMemoryPool }
import org.apache.kafka.common.memory. { MemoryPool , SimpleMemoryPool }
import org.apache.kafka.common.metrics._
import org.apache.kafka.common.metrics._
import org.apache.kafka.common.metrics.stats.Meter
import org.apache.kafka.common.metrics.stats.Meter
import org.apache.kafka.common.metrics.stats.Total
import org.apache.kafka.common.metrics.stats.Total
import org.apache.kafka.common.network.KafkaChannel.ChannelMuteEvent
import org.apache.kafka.common.network.KafkaChannel.ChannelMuteEvent
import org.apache.kafka.common.network. { ChannelBuilder , ChannelBuilders , KafkaChannel , ListenerName , Selectable , Send , Selector => KSelector }
import org.apache.kafka.common.network. { ChannelBuilder , ChannelBuilders , KafkaChannel , ListenerName , ListenerReconfigurable , Selectable , Send , Selector => KSelector }
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.protocol.ApiKeys
import org.apache.kafka.common.requests. { RequestContext , RequestHeader }
import org.apache.kafka.common.requests. { RequestContext , RequestHeader }
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.security.auth.SecurityProtocol
@ -70,7 +72,11 @@ import scala.util.control.ControlThrowable
* Acceptor has 1 Processor thread that has its own selector and read requests from the socket .
* Acceptor has 1 Processor thread that has its own selector and read requests from the socket .
* 1 Handler thread that handles requests and produce responses back to the processor thread for writing .
* 1 Handler thread that handles requests and produce responses back to the processor thread for writing .
*/
*/
class SocketServer ( val config : KafkaConfig , val metrics : Metrics , val time : Time , val credentialProvider : CredentialProvider ) extends Logging with KafkaMetricsGroup {
class SocketServer ( val config : KafkaConfig ,
val metrics : Metrics ,
val time : Time ,
val credentialProvider : CredentialProvider )
extends Logging with KafkaMetricsGroup with BrokerReconfigurable {
private val maxQueuedRequests = config . queuedMaxRequests
private val maxQueuedRequests = config . queuedMaxRequests
@ -109,7 +115,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time
*/
*/
def startup ( startupProcessors : Boolean = true ) {
def startup ( startupProcessors : Boolean = true ) {
this . synchronized {
this . synchronized {
connectionQuotas = new ConnectionQuotas ( config . maxConnectionsPerIp , config . maxConnectionsPerIpOverrides )
connectionQuotas = new ConnectionQuotas ( config , time )
createControlPlaneAcceptorAndProcessor ( config . controlPlaneListener )
createControlPlaneAcceptorAndProcessor ( config . controlPlaneListener )
createDataPlaneAcceptorsAndProcessors ( config . numNetworkThreads , config . dataPlaneListeners )
createDataPlaneAcceptorsAndProcessors ( config . numNetworkThreads , config . dataPlaneListeners )
if ( startupProcessors ) {
if ( startupProcessors ) {
@ -212,6 +218,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time
private def createDataPlaneAcceptorsAndProcessors ( dataProcessorsPerListener : Int ,
private def createDataPlaneAcceptorsAndProcessors ( dataProcessorsPerListener : Int ,
endpoints : Seq [ EndPoint ] ) : Unit = synchronized {
endpoints : Seq [ EndPoint ] ) : Unit = synchronized {
endpoints . foreach { endpoint =>
endpoints . foreach { endpoint =>
connectionQuotas . addListener ( config , endpoint . listenerName )
val dataPlaneAcceptor = createAcceptor ( endpoint , DataPlaneMetricPrefix )
val dataPlaneAcceptor = createAcceptor ( endpoint , DataPlaneMetricPrefix )
addDataPlaneProcessors ( dataPlaneAcceptor , endpoint , dataProcessorsPerListener )
addDataPlaneProcessors ( dataPlaneAcceptor , endpoint , dataProcessorsPerListener )
KafkaThread . nonDaemon ( s" data-plane-kafka-socket-acceptor- ${ endpoint . listenerName } - ${ endpoint . securityProtocol } - ${ endpoint . port } " , dataPlaneAcceptor ) . start ( )
KafkaThread . nonDaemon ( s" data-plane-kafka-socket-acceptor- ${ endpoint . listenerName } - ${ endpoint . securityProtocol } - ${ endpoint . port } " , dataPlaneAcceptor ) . start ( )
@ -223,6 +230,7 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time
private def createControlPlaneAcceptorAndProcessor ( endpointOpt : Option [ EndPoint ] ) : Unit = synchronized {
private def createControlPlaneAcceptorAndProcessor ( endpointOpt : Option [ EndPoint ] ) : Unit = synchronized {
endpointOpt . foreach { endpoint =>
endpointOpt . foreach { endpoint =>
connectionQuotas . addListener ( config , endpoint . listenerName )
val controlPlaneAcceptor = createAcceptor ( endpoint , ControlPlaneMetricPrefix )
val controlPlaneAcceptor = createAcceptor ( endpoint , ControlPlaneMetricPrefix )
val controlPlaneProcessor = newProcessor ( nextProcessorId , controlPlaneRequestChannelOpt . get , connectionQuotas , endpoint . listenerName , endpoint . securityProtocol , memoryPool )
val controlPlaneProcessor = newProcessor ( nextProcessorId , controlPlaneRequestChannelOpt . get , connectionQuotas , endpoint . listenerName , endpoint . securityProtocol , memoryPool )
controlPlaneAcceptorOpt = Some ( controlPlaneAcceptor )
controlPlaneAcceptorOpt = Some ( controlPlaneAcceptor )
@ -324,18 +332,33 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time
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 )
dataPlaneAcceptors . asScala . remove ( endpoint ) . foreach ( _ . shutdown ( ) )
dataPlaneAcceptors . asScala . remove ( endpoint ) . foreach ( _ . shutdown ( ) )
}
}
}
}
def updateMaxConnectionsPerIp ( maxConnectionsPerIp : Int ) : Unit = {
override def reconfigurableConfigs : Set [ String ] = SocketServer . ReconfigurableConfigs
info ( s" Updating maxConnectionsPerIp: $maxConnectionsPerIp " )
connectionQuotas . updateMaxConnectionsPerIp ( maxConnectionsPerIp )
override def validateReconfiguration ( newConfig : KafkaConfig ) : Unit = {
}
}
def updateMaxConnectionsPerIpOverride ( maxConnectionsPerIpOverrides : Map [ String , Int ] ) : Unit = {
override def reconfigure ( oldConfig : KafkaConfig , newConfig : KafkaConfig ) : Unit = {
info ( s" Updating maxConnectionsPerIpOverrides: ${ maxConnectionsPerIpOverrides . map { case ( k , v ) => s" $k = $v " } . mkString ( "," ) } " )
val maxConnectionsPerIp = newConfig . maxConnectionsPerIp
connectionQuotas . updateMaxConnectionsPerIpOverride ( maxConnectionsPerIpOverrides )
if ( maxConnectionsPerIp != oldConfig . maxConnectionsPerIp ) {
info ( s" Updating maxConnectionsPerIp: $maxConnectionsPerIp " )
connectionQuotas . updateMaxConnectionsPerIp ( maxConnectionsPerIp )
}
val maxConnectionsPerIpOverrides = newConfig . maxConnectionsPerIpOverrides
if ( maxConnectionsPerIpOverrides != oldConfig . maxConnectionsPerIpOverrides ) {
info ( s" Updating maxConnectionsPerIpOverrides: ${ maxConnectionsPerIpOverrides . map { case ( k , v ) => s" $k = $v " } . mkString ( "," ) } " )
connectionQuotas . updateMaxConnectionsPerIpOverride ( maxConnectionsPerIpOverrides )
}
val maxConnections = newConfig . maxConnections
if ( maxConnections != oldConfig . maxConnections ) {
info ( s" Updating broker-wide maxConnections: $maxConnections " )
connectionQuotas . updateBrokerMaxConnections ( maxConnections )
}
}
}
// `protected` for test usage
// `protected` for test usage
@ -373,6 +396,13 @@ object SocketServer {
val ControlPlaneThreadPrefix = "control-plane"
val ControlPlaneThreadPrefix = "control-plane"
val DataPlaneMetricPrefix = ""
val DataPlaneMetricPrefix = ""
val ControlPlaneMetricPrefix = "ControlPlane"
val ControlPlaneMetricPrefix = "ControlPlane"
val ReconfigurableConfigs = Set (
KafkaConfig . MaxConnectionsPerIpProp ,
KafkaConfig . MaxConnectionsPerIpOverridesProp ,
KafkaConfig . MaxConnectionsProp )
val ListenerReconfigurableConfigs = Set ( KafkaConfig . MaxConnectionsProp )
}
}
/* *
/* *
@ -427,10 +457,10 @@ private[kafka] abstract class AbstractServerThread(connectionQuotas: ConnectionQ
/* *
/* *
* Close `channel` and decrement the connection count .
* Close `channel` and decrement the connection count .
*/
*/
def close ( channel : SocketChannel ) : Unit = {
def close ( listenerName : ListenerName , channel : SocketChannel ) : Unit = {
if ( channel != null ) {
if ( channel != null ) {
debug ( s" Closing connection from ${ channel . socket . getRemoteSocketAddress ( ) } " )
debug ( s" Closing connection from ${ channel . socket . getRemoteSocketAddress ( ) } " )
connectionQuotas . dec ( channel . socket . getInetAddress )
connectionQuotas . dec ( listenerName , channel . socket . getInetAddress )
CoreUtils . swallow ( channel . socket ( ) . close ( ) , this , Level . ERROR )
CoreUtils . swallow ( channel . socket ( ) . close ( ) , this , Level . ERROR )
CoreUtils . swallow ( channel . close ( ) , this , Level . ERROR )
CoreUtils . swallow ( channel . close ( ) , this , Level . ERROR )
}
}
@ -500,6 +530,7 @@ private[kafka] class Acceptor(val endPoint: EndPoint,
var currentProcessorIndex = 0
var currentProcessorIndex = 0
while ( isRunning ) {
while ( isRunning ) {
try {
try {
val ready = nioSelector . select ( 500 )
val ready = nioSelector . select ( 500 )
if ( ready > 0 ) {
if ( ready > 0 ) {
val keys = nioSelector . selectedKeys ( )
val keys = nioSelector . selectedKeys ( )
@ -508,6 +539,7 @@ private[kafka] class Acceptor(val endPoint: EndPoint,
try {
try {
val key = iter . next
val key = iter . next
iter . remove ( )
iter . remove ( )
if ( key . isAcceptable ) {
if ( key . isAcceptable ) {
accept ( key ) . foreach { socketChannel =>
accept ( key ) . foreach { socketChannel =>
@ -582,7 +614,7 @@ private[kafka] class Acceptor(val endPoint: EndPoint,
val serverSocketChannel = key . channel ( ) . asInstanceOf [ ServerSocketChannel ]
val serverSocketChannel = key . channel ( ) . asInstanceOf [ ServerSocketChannel ]
val socketChannel = serverSocketChannel . accept ( )
val socketChannel = serverSocketChannel . accept ( )
try {
try {
connectionQuotas . inc ( socketChannel . socket ( ) . getInetAddress )
connectionQuotas . inc ( endPoint . listenerName , socketChannel . socket . getInetAddress , blockedPercentMeter )
socketChannel . configureBlocking ( false )
socketChannel . configureBlocking ( false )
socketChannel . socket ( ) . setTcpNoDelay ( true )
socketChannel . socket ( ) . setTcpNoDelay ( true )
socketChannel . socket ( ) . setKeepAlive ( true )
socketChannel . socket ( ) . setKeepAlive ( true )
@ -592,7 +624,7 @@ private[kafka] class Acceptor(val endPoint: EndPoint,
} 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. " )
close ( socketChannel )
close ( endPoint . listenerName , socketChannel )
None
None
}
}
}
}
@ -731,6 +763,7 @@ private[kafka] class Processor(val id: Int,
processCompletedReceives ( )
processCompletedReceives ( )
processCompletedSends ( )
processCompletedSends ( )
processDisconnected ( )
processDisconnected ( )
closeExcessConnections ( )
} catch {
} catch {
// We catch all the throwables here to prevent the processor thread from exiting . We do this because
// We catch all the throwables here to prevent the processor thread from exiting . We do this because
// letting a processor exit might cause a bigger impact on the broker . This behavior might need to be
// letting a processor exit might cause a bigger impact on the broker . This behavior might need to be
@ -911,13 +944,21 @@ private[kafka] class Processor(val id: Int,
} . remoteHost
} . remoteHost
inflightResponses . remove ( connectionId ) . foreach ( updateRequestMetrics )
inflightResponses . remove ( connectionId ) . foreach ( updateRequestMetrics )
// the channel has been closed by the selector but the quotas still need to be updated
// the channel has been closed by the selector but the quotas still need to be updated
connectionQuotas . dec ( InetAddress . getByName ( remoteHost ) )
connectionQuotas . dec ( listenerName , InetAddress . getByName ( remoteHost ) )
} catch {
} catch {
case e : Throwable => processException ( s" Exception while processing disconnection of $connectionId " , e )
case e : Throwable => processException ( s" Exception while processing disconnection of $connectionId " , e )
}
}
}
}
}
}
private def closeExcessConnections ( ) : Unit = {
if ( connectionQuotas . maxConnectionsExceeded ( listenerName ) ) {
val channel = selector . lowestPriorityChannel ( )
if ( channel != null )
close ( channel . id )
}
}
/* *
/* *
* Close the connection identified by `connectionId` and decrement the connection count .
* Close the connection identified by `connectionId` and decrement the connection count .
* The channel will be immediately removed from the selector 's `channels` or `closingChannels`
* The channel will be immediately removed from the selector 's `channels` or `closingChannels`
@ -930,7 +971,7 @@ private[kafka] class Processor(val id: Int,
debug ( s" Closing selector connection $connectionId " )
debug ( s" Closing selector connection $connectionId " )
val address = channel . socketAddress
val address = channel . socketAddress
if ( address != null )
if ( address != null )
connectionQuotas . dec ( address )
connectionQuotas . dec ( listenerName , address )
selector . close ( connectionId )
selector . close ( connectionId )
inflightResponses . remove ( connectionId ) . foreach ( response => updateRequestMetrics ( response ) )
inflightResponses . remove ( connectionId ) . foreach ( response => updateRequestMetrics ( response ) )
@ -977,7 +1018,7 @@ private[kafka] class Processor(val id: Int,
case e : Throwable =>
case e : Throwable =>
val remoteAddress = channel . socket . getRemoteSocketAddress
val remoteAddress = channel . socket . getRemoteSocketAddress
// need to close the channel here to avoid a socket leak .
// need to close the channel here to avoid a socket leak .
close ( channel )
close ( listenerName , channel )
processException ( s" Processor $id closed connection from $remoteAddress " , e )
processException ( s" Processor $id closed connection from $remoteAddress " , e )
}
}
}
}
@ -1058,31 +1099,72 @@ private[kafka] class Processor(val id: Int,
}
}
class ConnectionQuotas ( val defaultMax : Int , overrideQuotas : Map [ String , Int ] ) {
class ConnectionQuotas ( config : KafkaConfig , time : Time ) extends Logging {
@volatile private var defaultMaxConnectionsPerIp = defaultMax
@volatile private var defaultMaxConnectionsPerIp : Int = config . maxConnectionsPerIp
@volatile private var maxConnectionsPerIpOverrides = overrideQuotas . map { case ( host , count ) => ( InetAddress . getByName ( host ) , count ) }
@volatile private var maxConnectionsPerIpOverrides = config . maxConnectionsPerIpOverrides . map { case ( host , count ) => ( InetAddress . getByName ( host ) , count ) }
@volatile private var brokerMaxConnections = config . maxConnections
private val counts = mutable . Map [ InetAddress , Int ] ( )
private val counts = mutable . Map [ InetAddress , Int ] ( )
def inc ( address : InetAddress ) {
// Listener counts and configs are synchronized on `counts`
private val listenerCounts = mutable . Map [ ListenerName , Int ] ( )
private val maxConnectionsPerListener = mutable . Map [ ListenerName , ListenerConnectionQuota ] ( )
@volatile private var totalCount = 0
def inc ( listenerName : ListenerName , address : InetAddress , acceptorBlockedPercentMeter : com . yammer . metrics . core . Meter ) {
counts . synchronized {
counts . synchronized {
waitForConnectionSlot ( listenerName , acceptorBlockedPercentMeter )
val count = counts . getOrElseUpdate ( address , 0 )
val count = counts . getOrElseUpdate ( address , 0 )
counts . put ( address , count + 1 )
counts . put ( address , count + 1 )
totalCount += 1
if ( listenerCounts . contains ( listenerName ) ) {
listenerCounts . put ( listenerName , listenerCounts ( listenerName ) + 1 )
}
val max = maxConnectionsPerIpOverrides . getOrElse ( address , defaultMaxConnectionsPerIp )
val max = maxConnectionsPerIpOverrides . getOrElse ( address , defaultMaxConnectionsPerIp )
if ( count >= max )
if ( count >= max )
throw new TooManyConnectionsException ( address , max )
throw new TooManyConnectionsException ( address , max )
}
}
}
}
def updateMaxConnectionsPerIp ( maxConnectionsPerIp : Int ) : Unit = {
private [ network ] def updateMaxConnectionsPerIp ( maxConnectionsPerIp : Int ) : Unit = {
defaultMaxConnectionsPerIp = maxConnectionsPerIp
defaultMaxConnectionsPerIp = maxConnectionsPerIp
}
}
def updateMaxConnectionsPerIpOverride ( overrideQuotas : Map [ String , Int ] ) : Unit = {
private [ network ] def updateMaxConnectionsPerIpOverride ( overrideQuotas : Map [ String , Int ] ) : Unit = {
maxConnectionsPerIpOverrides = overrideQuotas . map { case ( host , count ) => ( InetAddress . getByName ( host ) , count ) }
maxConnectionsPerIpOverrides = overrideQuotas . map { case ( host , count ) => ( InetAddress . getByName ( host ) , count ) }
}
}
def dec ( address : InetAddress ) {
private [ network ] def updateBrokerMaxConnections ( maxConnections : Int ) : Unit = {
counts . synchronized {
brokerMaxConnections = maxConnections
counts . notifyAll ( )
}
}
private [ network ] def addListener ( config : KafkaConfig , listenerName : ListenerName ) : Unit = {
counts . synchronized {
if ( ! maxConnectionsPerListener . contains ( listenerName ) ) {
val newListenerQuota = new ListenerConnectionQuota ( counts , listenerName )
maxConnectionsPerListener . put ( listenerName , newListenerQuota )
listenerCounts . put ( listenerName , 0 )
config . addReconfigurable ( newListenerQuota )
}
counts . notifyAll ( )
}
}
private [ network ] def removeListener ( config : KafkaConfig , listenerName : ListenerName ) : Unit = {
counts . synchronized {
maxConnectionsPerListener . remove ( listenerName ) . foreach { listenerQuota =>
listenerCounts . remove ( listenerName )
counts . notifyAll ( ) // wake up any waiting acceptors to close cleanly
config . removeReconfigurable ( listenerQuota )
}
}
}
def dec ( listenerName : ListenerName , address : InetAddress ) {
counts . synchronized {
counts . synchronized {
val count = counts . getOrElse ( address ,
val count = counts . getOrElse ( address ,
throw new IllegalArgumentException ( s" Attempted to decrease connection count for address with no connections, address: $address " ) )
throw new IllegalArgumentException ( s" Attempted to decrease connection count for address with no connections, address: $address " ) )
@ -1090,6 +1172,19 @@ class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) {
counts . remove ( address )
counts . remove ( address )
else
else
counts . put ( address , count - 1 )
counts . put ( address , count - 1 )
if ( totalCount <= 0 )
error ( s" Attempted to decrease total connection count for broker with no connections " )
totalCount -= 1
if ( maxConnectionsPerListener . contains ( listenerName ) ) {
val listenerCount = listenerCounts ( listenerName )
if ( listenerCount == 0 )
error ( s" Attempted to decrease connection count for listener $listenerName with no connections " )
else
listenerCounts . put ( listenerName , listenerCount - 1 )
}
counts . notifyAll ( ) // wake up any acceptors waiting to process a new connection since listener connection limit was reached
}
}
}
}
@ -1097,6 +1192,72 @@ class ConnectionQuotas(val defaultMax: Int, overrideQuotas: Map[String, Int]) {
counts . getOrElse ( address , 0 )
counts . getOrElse ( address , 0 )
}
}
private def waitForConnectionSlot ( listenerName : ListenerName ,
acceptorBlockedPercentMeter : com . yammer . metrics . core . Meter ) : Unit = {
counts . synchronized {
if ( ! connectionSlotAvailable ( listenerName ) ) {
val startNs = time . nanoseconds
do {
counts . wait ( )
} while ( ! connectionSlotAvailable ( listenerName ) )
acceptorBlockedPercentMeter . mark ( time . nanoseconds - startNs )
}
}
}
// This is invoked in every poll iteration and we close one LRU connection in an iteration
// if necessary
def maxConnectionsExceeded ( listenerName : ListenerName ) : Boolean = {
totalCount > brokerMaxConnections && ! protectedListener ( listenerName )
}
private def connectionSlotAvailable ( listenerName : ListenerName ) : Boolean = {
if ( listenerCounts ( listenerName ) >= maxListenerConnections ( listenerName ) )
false
else if ( protectedListener ( listenerName ) )
true
else
totalCount < brokerMaxConnections
}
private def protectedListener ( listenerName : ListenerName ) : Boolean =
config . interBrokerListenerName == listenerName && config . listeners . size > 1
private def maxListenerConnections ( listenerName : ListenerName ) : Int =
maxConnectionsPerListener . get ( listenerName ) . map ( _ . maxConnections ) . getOrElse ( Int . MaxValue )
class ListenerConnectionQuota ( lock : Object , listener : ListenerName ) extends ListenerReconfigurable {
@volatile private var _maxConnections = Int . MaxValue
private val listenerPropName = s" ${ listener . configPrefix } ${ KafkaConfig . MaxConnectionsProp } "
def maxConnections : Int = _maxConnections
override def listenerName ( ) : ListenerName = listener
override def configure ( configs : util.Map [ String , _ ] ) : Unit = {
_maxConnections = maxConnections ( configs )
}
override def reconfigurableConfigs ( ) : util.Set [ String ] = {
SocketServer . ListenerReconfigurableConfigs . asJava
}
override def validateReconfiguration ( configs : util.Map [ String , _ ] ) : Unit = {
val value = maxConnections ( configs )
if ( value <= 0 )
throw new ConfigException ( "Invalid max.connections $listenerMax" )
}
override def reconfigure ( configs : util.Map [ String , _ ] ) : Unit = {
lock . synchronized {
_maxConnections = maxConnections ( configs )
lock . notifyAll ( )
}
}
private def maxConnections ( configs : util.Map [ String , _ ] ) : Int = {
Option ( configs . get ( KafkaConfig . MaxConnectionsProp ) ) . map ( _ . toString . toInt ) . getOrElse ( Int . MaxValue )
}
}
}
}
class TooManyConnectionsException ( val ip : InetAddress , val count : Int ) extends KafkaException ( s" Too many connections from $ip (maximum = $count ) " )
class TooManyConnectionsException ( val ip : InetAddress , val count : Int ) extends KafkaException ( s" Too many connections from $ip (maximum = $count ) " )