From b7bbdffb5cba5bc15a1d63638082190a7624a14f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 01:24:51 +0800 Subject: [PATCH 01/20] remove dynamic Config related --- core/src/main/scala/kafka/server/ConfigHandler.scala | 4 ++-- core/src/main/scala/kafka/server/DynamicConfig.scala | 12 +++++------- .../server/metadata/DynamicConfigPublisher.scala | 4 ++-- 3 files changed, 9 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index aba4f9f77672d..0892afcfc6f0e 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -21,7 +21,7 @@ import java.util.{Collections, Properties} import kafka.log.UnifiedLog import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Logging -import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.QuotaConfig import org.apache.kafka.common.metrics.Quota._ import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.server.ClientMetricsManager @@ -146,7 +146,7 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager, class BrokerConfigHandler(private val brokerConfig: KafkaConfig, private val quotaManagers: QuotaManagers) extends ConfigHandler with Logging { def processConfigChanges(brokerId: String, properties: Properties): Unit = { - if (brokerId == ZooKeeperInternals.DEFAULT_STRING) + if (brokerId.isEmpty) brokerConfig.dynamicConfig.updateDefaultConfig(properties) else if (brokerConfig.brokerId == brokerId.trim.toInt) { brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties) diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index 988c567f1cfe7..f6452de676fad 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -23,7 +23,7 @@ import java.net.{InetAddress, UnknownHostException} import java.util.Properties import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.coordinator.group.GroupConfig -import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.QuotaConfig import java.util import scala.jdk.CollectionConverters._ @@ -85,12 +85,10 @@ object DynamicConfig { def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false) def isValidIpEntity(ip: String): Boolean = { - if (ip != ZooKeeperInternals.DEFAULT_STRING) { - try { - InetAddress.getByName(ip) - } catch { - case _: UnknownHostException => return false - } + try { + InetAddress.getByName(ip) + } catch { + case _: UnknownHostException => return false } true } diff --git a/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala index 763aa387962ee..6904921fb0cfa 100644 --- a/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala @@ -24,7 +24,7 @@ import kafka.utils.Logging import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, CLIENT_METRICS, GROUP, TOPIC} import org.apache.kafka.image.loader.LoaderManifest import org.apache.kafka.image.{MetadataDelta, MetadataImage} -import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals} +import org.apache.kafka.server.config.ConfigType import org.apache.kafka.server.fault.FaultHandler @@ -78,7 +78,7 @@ class DynamicConfigPublisher( // These are stored in KRaft with an empty name field. info("Updating cluster configuration : " + toLoggableProps(resource, props).mkString(",")) - nodeConfigHandler.processConfigChanges(ZooKeeperInternals.DEFAULT_STRING, props) + nodeConfigHandler.processConfigChanges(resource.name(), props) } catch { case t: Throwable => faultHandler.handleFault("Error updating " + s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + From a022b0ca9a18bd042471a34d733e35cb388dd3a7 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 02:21:46 +0800 Subject: [PATCH 02/20] fix some test, and revert some wrong approach --- .../kafka/server/ClientQuotaManager.scala | 27 +- .../metadata/ClientQuotaMetadataManager.scala | 17 +- .../kafka/server/ClientQuotaManagerTest.scala | 239 +++++++++++++++--- .../server/ClientQuotasRequestTest.scala | 4 +- .../ClientRequestQuotaManagerTest.scala | 9 +- .../ControllerMutationQuotaManagerTest.scala | 24 +- .../server/config/ZooKeeperInternals.java | 27 -- 7 files changed, 249 insertions(+), 98 deletions(-) delete mode 100644 server-common/src/main/java/org/apache/kafka/server/config/ZooKeeperInternals.java diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index f621d0dbcf437..ec210d3bd61fa 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -29,7 +29,7 @@ import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.metrics.stats.{Avg, CumulativeSum, Rate} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.{Sanitizer, Time} -import org.apache.kafka.server.config.{ClientQuotaManagerConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.ClientQuotaManagerConfig import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaEntity, ClientQuotaType, QuotaType, QuotaUtils, SensorAccess, ThrottleCallback, ThrottledChannel} import org.apache.kafka.server.util.ShutdownableThread import org.apache.kafka.network.Session @@ -55,7 +55,7 @@ object QuotaTypes { object ClientQuotaManager { // Purge sensors after 1 hour of inactivity val InactiveSensorExpirationTimeSeconds = 3600 - + val DefaultString = "" val DefaultClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity)) val DefaultUserQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None) val DefaultUserClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity)) @@ -76,13 +76,13 @@ object ClientQuotaManager { case object DefaultUserEntity extends BaseUserEntity { override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_USER - override def name: String = ZooKeeperInternals.DEFAULT_STRING + override def name: String = DefaultString override def toString: String = "default user" } case object DefaultClientIdEntity extends ClientQuotaEntity.ConfigEntity { override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_CLIENT_ID - override def name: String = ZooKeeperInternals.DEFAULT_STRING + override def name: String = DefaultString override def toString: String = "default client-id" } @@ -93,7 +93,7 @@ object ClientQuotaManager { def sanitizedUser: String = userEntity.map { case entity: UserEntity => entity.sanitizedUser - case DefaultUserEntity => ZooKeeperInternals.DEFAULT_STRING + case DefaultUserEntity => DefaultString }.getOrElse("") def clientId: String = clientIdEntity.map(_.name).getOrElse("") @@ -408,7 +408,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * @param sanitizedClientId sanitized client ID to override if quota applies to or * @param quota custom quota to apply or None if quota override is being removed */ - def updateQuota(sanitizedUser: Option[String], clientId: Option[String], sanitizedClientId: Option[String], quota: Option[Quota]): Unit = { + def updateQuota(sanitizedUser: Option[BaseUserEntity], clientId: Option[String], sanitizedClientId: Option[String], quota: Option[Quota]): Unit = { /* * Acquire the write lock to apply changes in the quota objects. * This method changes the quota in the overriddenQuota map and applies the update on the actual KafkaMetric object (if it exists). @@ -418,12 +418,10 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val userEntity = sanitizedUser.map { - case ZooKeeperInternals.DEFAULT_STRING => DefaultUserEntity - case user => UserEntity(user) - } + val userEntity = getOrDefaultUser(sanitizedUser) + val clientIdEntity = sanitizedClientId.map { - case ZooKeeperInternals.DEFAULT_STRING => DefaultClientIdEntity + case DefaultString => DefaultClientIdEntity case _ => ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided"))) } val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) @@ -451,6 +449,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } } + private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { + if (sanitizedUser.nonEmpty && sanitizedUser.get.name() == DefaultString) + Some(DefaultUserEntity) + else + sanitizedUser + } + /** * Updates metrics configs. This is invoked when quota configs are updated in ZooKeeper * or when partitions leaders change and custom callbacks that implement partition-based quotas diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index a6d3133f18697..ccf8d161a22a1 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -18,6 +18,7 @@ package kafka.server.metadata import kafka.network.ConnectionQuotas +import kafka.server.ClientQuotaManager import kafka.server.QuotaFactory.QuotaManagers import kafka.utils.Logging import org.apache.kafka.common.metrics.Quota @@ -26,7 +27,7 @@ import org.apache.kafka.common.utils.Sanitizer import java.net.{InetAddress, UnknownHostException} import org.apache.kafka.image.{ClientQuotaDelta, ClientQuotasDelta} -import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.QuotaConfig import scala.jdk.OptionConverters.RichOptionalDouble @@ -146,14 +147,14 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag // Convert entity into Options with sanitized values for QuotaManagers val (sanitizedUser, sanitizedClientId) = quotaEntity match { - case UserEntity(user) => (Some(Sanitizer.sanitize(user)), None) - case DefaultUserEntity => (Some(ZooKeeperInternals.DEFAULT_STRING), None) + case UserEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) + case DefaultUserEntity => (Some(ClientQuotaManager.DefaultUserEntity), None) case ClientIdEntity(clientId) => (None, Some(Sanitizer.sanitize(clientId))) - case DefaultClientIdEntity => (None, Some(ZooKeeperInternals.DEFAULT_STRING)) - case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(Sanitizer.sanitize(user)), Some(Sanitizer.sanitize(clientId))) - case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(ZooKeeperInternals.DEFAULT_STRING)) - case DefaultUserExplicitClientIdEntity(clientId) => (Some(ZooKeeperInternals.DEFAULT_STRING), Some(Sanitizer.sanitize(clientId))) - case DefaultUserDefaultClientIdEntity => (Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING)) + case DefaultClientIdEntity => (None, Some(ClientQuotaManager.DefaultString)) + case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(Sanitizer.sanitize(clientId))) + case ExplicitUserDefaultClientIdEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultString)) + case DefaultUserExplicitClientIdEntity(clientId) => (Some(ClientQuotaManager.DefaultUserEntity), Some(Sanitizer.sanitize(clientId))) + case DefaultUserDefaultClientIdEntity => (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultString)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 4a4107226dcfb..72199502dd5cd 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -20,7 +20,7 @@ import java.net.InetAddress import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.utils.Sanitizer -import org.apache.kafka.server.config.{ClientQuotaManagerConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.ClientQuotaManagerConfig import org.apache.kafka.network.Session import org.apache.kafka.server.quota.QuotaType import org.junit.jupiter.api.Assertions._ @@ -34,8 +34,18 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Case 1: Update the quota. Assert that the new quota value is returned - clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(2000, true))) - clientQuotaManager.updateQuota(client2.configUser, client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true))) + clientQuotaManager.updateQuota( + client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), + client1.configClientId, + client1.sanitizedConfigClientId, + Some(new Quota(2000, true)) + ) + clientQuotaManager.updateQuota( + client2.configUser.map(s => ClientQuotaManager.UserEntity(s)), + client2.configClientId, + client2.sanitizedConfigClientId, + Some(new Quota(4000, true)) + ) assertEquals(Long.MaxValue.toDouble, clientQuotaManager.quota(randomClient.user, randomClient.clientId).bound, 0.0, "Default producer quota should be " + Long.MaxValue.toDouble) @@ -50,22 +60,42 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 2: Change quota again. The quota should be updated within KafkaMetrics as well since the sensor was created. // p1 should not longer be throttled after the quota change - clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(3000, true))) + clientQuotaManager.updateQuota( + client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), + client1.configClientId, + client1.sanitizedConfigClientId, + Some(new Quota(3000, true)) + ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 0) assertEquals(0, throttleTimeMs, s"throttleTimeMs should be 0. was $throttleTimeMs") // Case 3: Change quota back to default. Should be throttled again - clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(500, true))) + clientQuotaManager.updateQuota( + client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), + client1.configClientId, + client1.sanitizedConfigClientId, + Some(new Quota(500, true)) + ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 0) assertTrue(throttleTimeMs > 0, s"throttleTimeMs should be > 0. was $throttleTimeMs") // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled - clientQuotaManager.updateQuota(client1.configUser, client1.configClientId, client1.sanitizedConfigClientId, None) - clientQuotaManager.updateQuota(defaultConfigClient.configUser, defaultConfigClient.configClientId, defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true))) + clientQuotaManager.updateQuota( + client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), + client1.configClientId, + client1.sanitizedConfigClientId, + None + ) + clientQuotaManager.updateQuota( + defaultConfigClient.configUser.map(s => ClientQuotaManager.UserEntity(s)), + defaultConfigClient.configClientId, + defaultConfigClient.sanitizedConfigClientId, + Some(new Quota(4000, true)) + ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") throttleTimeMs = maybeRecord(clientQuotaManager, client1.user, client1.clientId, 1000 * config.numQuotaSamples) @@ -85,7 +115,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client1 = UserClient("ANONYMOUS", "p1", None, Some("p1")) val client2 = UserClient("ANONYMOUS", "p2", None, Some("p2")) val randomClient = UserClient("ANONYMOUS", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", None, Some(ZooKeeperInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultString)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -98,7 +128,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client1 = UserClient("User1", "p1", Some("User1"), None) val client2 = UserClient("User2", "p2", Some("User2"), None) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ZooKeeperInternals.DEFAULT_STRING), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -112,7 +142,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client1 = UserClient("User1", "p1", Some("User1"), Some("p1")) val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -125,7 +155,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client1 = UserClient("User1", "p1", Some("User1"), None) val client2 = UserClient("User2", "p2", Some("User2"), None) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ZooKeeperInternals.DEFAULT_STRING), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -137,7 +167,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val client1 = UserClient("User1", "p1", Some("User1"), Some("p1")) val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -168,7 +198,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { assertEquals(Double.MaxValue, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) // Set default quota config - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), None, None, Some(new Quota(10, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + None, + None, + Some(new Quota(10, true)) + ) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) } finally { clientQuotaManager.shutdown() @@ -186,11 +221,21 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) // Set default quota config - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), None, None, Some(new Quota(10, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + None, + None, + Some(new Quota(10, true)) + ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) // Remove default quota config, back to no quotas - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), None, None, None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + None, + None, + None + ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) } finally { clientQuotaManager.shutdown() @@ -205,11 +250,21 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Set quota config - clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(10, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + None, + None, + Some(new Quota(10, true)) + ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) // Remove quota config, back to no quotas - clientQuotaManager.updateQuota(Some("userA"), None, None, None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + None, + None, + None + ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) } finally { clientQuotaManager.shutdown() @@ -224,11 +279,21 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Set quota config - clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), + Some("client1"), + Some(new Quota(10, true)) + ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) // Remove quota config, back to no quotas - clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), + Some("client1"), + None + ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) } finally { clientQuotaManager.shutdown() @@ -241,16 +306,66 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), None, None, Some(new Quota(1000, true))) - clientQuotaManager.updateQuota(None, Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(new Quota(2000, true))) - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(new Quota(3000, true))) - clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(4000, true))) - clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(5000, true))) - clientQuotaManager.updateQuota(Some("userB"), None, None, Some(new Quota(6000, true))) - clientQuotaManager.updateQuota(Some("userB"), Some("client1"), Some("client1"), Some(new Quota(7000, true))) - clientQuotaManager.updateQuota(Some("userB"), Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(new Quota(8000, true))) - clientQuotaManager.updateQuota(Some("userC"), None, None, Some(new Quota(10000, true))) - clientQuotaManager.updateQuota(None, Some("client1"), Some("client1"), Some(new Quota(9000, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + None, + None, + Some(new Quota(1000, true)) + ) + clientQuotaManager.updateQuota( + None, + Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.DefaultString), + Some(new Quota(2000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.DefaultString), + Some(new Quota(3000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + None, + None, + Some(new Quota(4000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), + Some("client1"), + Some(new Quota(5000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userB")), + None, + None, + Some(new Quota(6000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userB")), + Some("client1"), + Some("client1"), + Some(new Quota(7000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userB")), + Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.DefaultString), + Some(new Quota(8000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userC")), + None, + None, + Some(new Quota(10000, true)) + ) + clientQuotaManager.updateQuota( + None, + Some("client1"), + Some("client1"), + Some(new Quota(9000, true)) + ) checkQuota(clientQuotaManager, "userA", "client1", 5000, 4500, expectThrottle = false) // quota takes precedence over checkQuota(clientQuotaManager, "userA", "client2", 4000, 4500, expectThrottle = true) // quota takes precedence over and defaults @@ -266,32 +381,72 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userE", "client1", 3000, 2500, expectThrottle = false) // Remove default quota config, revert to default - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.DefaultString), + None + ) checkQuota(clientQuotaManager, "userD", "client1", 1000, 0, expectThrottle = false) // Metrics tags changed, restart counter checkQuota(clientQuotaManager, "userE", "client4", 1000, 1500, expectThrottle = true) checkQuota(clientQuotaManager, "userF", "client4", 1000, 800, expectThrottle = false) // Default quota shared across clients of user checkQuota(clientQuotaManager, "userF", "client5", 1000, 800, expectThrottle = true) // Remove default quota config, revert to default - clientQuotaManager.updateQuota(Some(ZooKeeperInternals.DEFAULT_STRING), None, None, None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + None, + None, + None + ) checkQuota(clientQuotaManager, "userF", "client4", 2000, 0, expectThrottle = false) // Default quota shared across client-id of all users checkQuota(clientQuotaManager, "userF", "client5", 2000, 0, expectThrottle = false) checkQuota(clientQuotaManager, "userF", "client5", 2000, 2500, expectThrottle = true) checkQuota(clientQuotaManager, "userG", "client5", 2000, 0, expectThrottle = true) // Update quotas - clientQuotaManager.updateQuota(Some("userA"), None, None, Some(new Quota(8000, true))) - clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), Some(new Quota(10000, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + None, + None, + Some(new Quota(8000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), + Some("client1"), + Some(new Quota(10000, true)) + ) checkQuota(clientQuotaManager, "userA", "client2", 8000, 0, expectThrottle = false) checkQuota(clientQuotaManager, "userA", "client2", 8000, 4500, expectThrottle = true) // Throttled due to sum of new and earlier values checkQuota(clientQuotaManager, "userA", "client1", 10000, 0, expectThrottle = false) checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, expectThrottle = true) - clientQuotaManager.updateQuota(Some("userA"), Some("client1"), Some("client1"), None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), + Some("client1"), + None + ) checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, expectThrottle = true) // Throttled due to shared user quota - clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), Some(new Quota(11000, true))) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client6"), + Some("client6"), + Some(new Quota(11000, true)) + ) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) - clientQuotaManager.updateQuota(Some("userA"), Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), Some(new Quota(12000, true))) - clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.DefaultString), + Some(new Quota(12000, true)) + ) + clientQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("userA")), + Some("client6"), + Some("client6"), + None + ) checkQuota(clientQuotaManager, "userA", "client6", 12000, 4000, expectThrottle = true) // Throttled due to sum of new and earlier values } finally { @@ -304,7 +459,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Produce", "")) try { - clientQuotaManager.updateQuota(None, Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) // We have 10 second windows. Make sure that there is no quota violation @@ -352,7 +507,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireThrottleTimeSensor(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) @@ -374,7 +529,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireQuotaSensors(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) @@ -401,7 +556,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") val clientId = "client@#$%" try { - clientQuotaManager.updateQuota(None, Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", clientId, 100) @@ -421,6 +576,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // The class under test expects only sanitized client configs. We pass both the default value (which should not be // sanitized to ensure it remains unique) and non-default values, so we need to take care in generating the sanitized // client ID - def sanitizedConfigClientId = configClientId.map(x => if (x == ZooKeeperInternals.DEFAULT_STRING) ZooKeeperInternals.DEFAULT_STRING else Sanitizer.sanitize(x)) + def sanitizedConfigClientId = configClientId.map(x => if (x == ClientQuotaManager.DefaultString) ClientQuotaManager.DefaultString else Sanitizer.sanitize(x)) } } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index cec9289e69140..9dadefd72ab39 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -28,7 +28,7 @@ import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQuotasResponse, DescribeClientQuotasRequest, DescribeClientQuotasResponse} import org.apache.kafka.common.test.ClusterInstance -import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} +import org.apache.kafka.server.config.QuotaConfig import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Disabled @@ -521,7 +521,7 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { def testClientQuotasWithDefaultName(): Unit = { // An entity using the name associated with the default entity name. The entity's name should be sanitized so // that it does not conflict with the default entity name. - val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.CLIENT_ID -> ZooKeeperInternals.DEFAULT_STRING).asJava) + val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.CLIENT_ID -> ClientQuotaManager.DefaultString).asJava) alterEntityQuotas(entity, Map(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> Some(20000.0)), validateOnly = false) verifyDescribeEntityQuotas(entity, Map(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 20000.0)) diff --git a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala index 2b5471653d9a9..acbb769e74f84 100644 --- a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala @@ -30,11 +30,16 @@ class ClientRequestQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testRequestPercentageQuotaViolation(): Unit = { val clientRequestQuotaManager = new ClientRequestQuotaManager(config, metrics, time, "", Optional.empty()) - clientRequestQuotaManager.updateQuota(Some("ANONYMOUS"), Some("test-client"), Some("test-client"), Some(Quota.upperBound(1))) + clientRequestQuotaManager.updateQuota( + Some(ClientQuotaManager.UserEntity("ANONYMOUS")), + Some("test-client"), + Some("test-client"), + Some(Quota.upperBound(1)) + ) val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", QuotaType.REQUEST.toString, "")) def millisToPercent(millis: Double) = millis * 1000 * 1000 * ClientRequestQuotaManager.NANOS_TO_PERCENTAGE_PER_SECOND try { - // We have 10 second windows. Make sure that there is no quota violation + // We have 10 seconds windows. Make sure that there is no quota violation // if we are under the quota for (_ <- 0 until 10) { assertEquals(0, maybeRecord(clientRequestQuotaManager, "ANONYMOUS", "test-client", millisToPercent(4))) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala index 8fa4a290cc7a8..ba095cf47a84e 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala @@ -145,8 +145,12 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testControllerMutationQuotaViolation(): Unit = { withQuotaManager { quotaManager => - quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId), - Some(Quota.upperBound(10))) + quotaManager.updateQuota( + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), + Some(ClientId), + Some(Quota.upperBound(10)) + ) val queueSizeMetric = metrics.metrics().get( metrics.metricName("queue-size", QuotaType.CONTROLLER_MUTATION.toString, "")) @@ -204,8 +208,12 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testNewStrictQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = { withQuotaManager { quotaManager => - quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId), - Some(Quota.upperBound(10))) + quotaManager.updateQuota( + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), + Some(ClientId), + Some(Quota.upperBound(10)) + ) val quota = quotaManager.newStrictQuotaFor(buildSession(User), ClientId) assertTrue(quota.isInstanceOf[StrictControllerMutationQuota]) @@ -223,8 +231,12 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { @Test def testNewPermissiveQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = { withQuotaManager { quotaManager => - quotaManager.updateQuota(Some(User), Some(ClientId), Some(ClientId), - Some(Quota.upperBound(10))) + quotaManager.updateQuota( + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), + Some(ClientId), + Some(Quota.upperBound(10)) + ) val quota = quotaManager.newPermissiveQuotaFor(buildSession(User), ClientId) assertTrue(quota.isInstanceOf[PermissiveControllerMutationQuota]) } diff --git a/server-common/src/main/java/org/apache/kafka/server/config/ZooKeeperInternals.java b/server-common/src/main/java/org/apache/kafka/server/config/ZooKeeperInternals.java deleted file mode 100644 index 72ac563ae52f8..0000000000000 --- a/server-common/src/main/java/org/apache/kafka/server/config/ZooKeeperInternals.java +++ /dev/null @@ -1,27 +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.server.config; - -public class ZooKeeperInternals { - /** - * This string is used in ZooKeeper in several places to indicate a default entity type. - * For example, default user quotas are stored under /config/users/<default> - * Note that AdminClient does not use this to indicate a default, nor do records in KRaft mode. - * This constant will go away in Apache Kafka 4.0 with the end of ZK mode. - */ - public static final String DEFAULT_STRING = ""; -} From 6cef55fb6d3f9e87a28a17f267f4405c0d46008c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 10:29:35 +0800 Subject: [PATCH 03/20] update the function signature --- .../kafka/server/ClientQuotaManager.scala | 23 ++-- .../metadata/ClientQuotaMetadataManager.scala | 28 +++-- .../DynamicBrokerReconfigurationTest.scala | 7 +- .../kafka/server/ClientQuotaManagerTest.scala | 119 ++++++++---------- .../ClientRequestQuotaManagerTest.scala | 5 +- .../ControllerMutationQuotaManagerTest.scala | 15 +-- 6 files changed, 97 insertions(+), 100 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index ec210d3bd61fa..d588c4b9e083c 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -260,7 +260,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ def unrecordQuotaSensor(request: RequestChannel.Request, value: Double, timeMs: Long): Unit = { val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId) - clientSensors.quotaSensor.record(value * (-1), timeMs, false) + clientSensors.quotaSensor.record(value * -1, timeMs, false) } /** @@ -404,11 +404,14 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * for any of these levels. * * @param sanitizedUser user to override if quota applies to or - * @param clientId client to override if quota applies to or * @param sanitizedClientId sanitized client ID to override if quota applies to or * @param quota custom quota to apply or None if quota override is being removed */ - def updateQuota(sanitizedUser: Option[BaseUserEntity], clientId: Option[String], sanitizedClientId: Option[String], quota: Option[Quota]): Unit = { + def updateQuota( + sanitizedUser: Option[BaseUserEntity], + sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], + quota: Option[Quota] + ): Unit = { /* * Acquire the write lock to apply changes in the quota objects. * This method changes the quota in the overriddenQuota map and applies the update on the actual KafkaMetric object (if it exists). @@ -419,11 +422,8 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, lock.writeLock().lock() try { val userEntity = getOrDefaultUser(sanitizedUser) + val clientIdEntity = getOrDefaultClient(sanitizedClientId) - val clientIdEntity = sanitizedClientId.map { - case DefaultString => DefaultClientIdEntity - case _ => ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided"))) - } val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) if (userEntity.nonEmpty) { @@ -449,6 +449,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } } + private def getOrDefaultClient( + sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] + ): Option[ClientQuotaEntity.ConfigEntity] = { + if (sanitizedClientId.nonEmpty && sanitizedClientId.get.name() == DefaultString) + Some(DefaultClientIdEntity) + else + sanitizedClientId + } + private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { if (sanitizedUser.nonEmpty && sanitizedUser.get.name() == DefaultString) Some(DefaultUserEntity) diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index ccf8d161a22a1..49994189ae267 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -147,14 +147,22 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag // Convert entity into Options with sanitized values for QuotaManagers val (sanitizedUser, sanitizedClientId) = quotaEntity match { - case UserEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) - case DefaultUserEntity => (Some(ClientQuotaManager.DefaultUserEntity), None) - case ClientIdEntity(clientId) => (None, Some(Sanitizer.sanitize(clientId))) - case DefaultClientIdEntity => (None, Some(ClientQuotaManager.DefaultString)) - case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(Sanitizer.sanitize(clientId))) - case ExplicitUserDefaultClientIdEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultString)) - case DefaultUserExplicitClientIdEntity(clientId) => (Some(ClientQuotaManager.DefaultUserEntity), Some(Sanitizer.sanitize(clientId))) - case DefaultUserDefaultClientIdEntity => (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultString)) + case UserEntity(user) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) + case DefaultUserEntity => + (Some(ClientQuotaManager.DefaultUserEntity), None) + case ClientIdEntity(clientId) => + (None, Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case DefaultClientIdEntity => + (None, Some(ClientQuotaManager.DefaultClientIdEntity)) + case ExplicitUserExplicitClientIdEntity(user, clientId) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case ExplicitUserDefaultClientIdEntity(user) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultClientIdEntity)) + case DefaultUserExplicitClientIdEntity(clientId) => + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case DefaultUserDefaultClientIdEntity => + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } @@ -162,9 +170,9 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag try { manager.updateQuota( sanitizedUser = sanitizedUser, - clientId = sanitizedClientId.map(Sanitizer.desanitize), sanitizedClientId = sanitizedClientId, - quota = quotaValue) + quota = quotaValue + ) } catch { case t: Throwable => error(s"Failed to update user-client quota $quotaEntity", t) } diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index 49a0ebc21f4bf..f4de50d7cd8a3 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -934,8 +934,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup // non-default value to trigger a new metric val clientId = "test-client-1" servers.foreach { server => - server.quotaManagers.produce.updateQuota(None, Some(clientId), Some(clientId), - Some(Quota.upperBound(10000000))) + server.quotaManagers.produce.updateQuota( + None, + Some(ClientQuotaManager.ClientIdEntity(clientId)), + Some(Quota.upperBound(10000000)) + ) } val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol, clientId) TestUtils.waitUntilTrue(() => consumerThread.received >= 5, "Messages not sent") diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 72199502dd5cd..67d1dae6f82d6 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -36,14 +36,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.configClientId, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( client2.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client2.configClientId, - client2.sanitizedConfigClientId, + client2.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), Some(new Quota(4000, true)) ) @@ -62,8 +60,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.configClientId, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), Some(new Quota(3000, true)) ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") @@ -74,8 +71,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.configClientId, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), Some(new Quota(500, true)) ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") @@ -86,14 +82,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.configClientId, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), None ) clientQuotaManager.updateQuota( defaultConfigClient.configUser.map(s => ClientQuotaManager.UserEntity(s)), - defaultConfigClient.configClientId, - defaultConfigClient.sanitizedConfigClientId, + defaultConfigClient.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), Some(new Quota(4000, true)) ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") @@ -201,7 +195,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), None, - None, Some(new Quota(10, true)) ) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) @@ -224,7 +217,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), None, - None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -233,7 +225,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), None, - None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -253,7 +244,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -262,7 +252,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -281,17 +270,15 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Set quota config clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) // Remove quota config, back to no quotas clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.ClientIdEntity("client1")), None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -309,61 +296,51 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), None, - None, Some(new Quota(1000, true)) ) clientQuotaManager.updateQuota( - None, - Some(ClientQuotaManager.DefaultString), - Some(ClientQuotaManager.DefaultString), + None, + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), - Some(ClientQuotaManager.DefaultString), - Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), Some(new Quota(3000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(4000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(5000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), None, - None, Some(new Quota(6000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(7000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userB")), - Some(ClientQuotaManager.DefaultString), - Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.UserEntity("userB")), + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userC")), None, - None, Some(new Quota(10000, true)) ) clientQuotaManager.updateQuota( None, - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(9000, true)) ) @@ -382,9 +359,8 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, revert to default clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), - Some(ClientQuotaManager.DefaultString), - Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), None ) checkQuota(clientQuotaManager, "userD", "client1", 1000, 0, expectThrottle = false) // Metrics tags changed, restart counter @@ -396,7 +372,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), None, - None, None ) checkQuota(clientQuotaManager, "userF", "client4", 2000, 0, expectThrottle = false) // Default quota shared across client-id of all users @@ -408,13 +383,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10000, true)) ) checkQuota(clientQuotaManager, "userA", "client2", 8000, 0, expectThrottle = false) @@ -423,28 +396,24 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, expectThrottle = true) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), - Some("client1"), + Some(ClientQuotaManager.ClientIdEntity("client1")), None ) checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, expectThrottle = true) // Throttled due to shared user quota clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), - Some("client6"), - Some("client6"), + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.ClientIdEntity("client6")), Some(new Quota(11000, true)) ) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), - Some(ClientQuotaManager.DefaultString), - Some(ClientQuotaManager.DefaultString), + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), Some(new Quota(12000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), - Some("client6"), - Some("client6"), + Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.ClientIdEntity("client6")), None ) checkQuota(clientQuotaManager, "userA", "client6", 12000, 4000, expectThrottle = true) // Throttled due to sum of new and earlier values @@ -459,10 +428,13 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", "Produce", "")) try { - clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), - Some(new Quota(500, true))) + clientQuotaManager.updateQuota( + None, + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(new Quota(500, true)) + ) - // We have 10 second windows. Make sure that there is no quota violation + // We have 10 seconds windows. Make sure that there is no quota violation // if we produce under the quota for (_ <- 0 until 10) { assertEquals(0, maybeRecord(clientQuotaManager, "ANONYMOUS", "unknown", 400)) @@ -507,8 +479,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireThrottleTimeSensor(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), - Some(new Quota(500, true))) + clientQuotaManager.updateQuota( + None, + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(new Quota(500, true)) + ) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) // remove the throttle time sensor @@ -529,8 +504,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireQuotaSensors(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), - Some(new Quota(500, true))) + clientQuotaManager.updateQuota( + None, + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(new Quota(500, true)) + ) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) // remove all the sensors @@ -556,8 +534,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") val clientId = "client@#$%" try { - clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), - Some(new Quota(500, true))) + clientQuotaManager.updateQuota( + None, + Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(new Quota(500, true)) + ) maybeRecord(clientQuotaManager, "ANONYMOUS", clientId, 100) diff --git a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala index acbb769e74f84..368280d235453 100644 --- a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala @@ -31,9 +31,8 @@ class ClientRequestQuotaManagerTest extends BaseClientQuotaManagerTest { def testRequestPercentageQuotaViolation(): Unit = { val clientRequestQuotaManager = new ClientRequestQuotaManager(config, metrics, time, "", Optional.empty()) clientRequestQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("ANONYMOUS")), - Some("test-client"), - Some("test-client"), + Some(ClientQuotaManager.UserEntity("ANONYMOUS")), + Some(ClientQuotaManager.ClientIdEntity("test-client")), Some(Quota.upperBound(1)) ) val queueSizeMetric = metrics.metrics().get(metrics.metricName("queue-size", QuotaType.REQUEST.toString, "")) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala index ba095cf47a84e..2a7cfe35a85b0 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala @@ -146,9 +146,8 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { def testControllerMutationQuotaViolation(): Unit = { withQuotaManager { quotaManager => quotaManager.updateQuota( - Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), - Some(ClientId), + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) val queueSizeMetric = metrics.metrics().get( @@ -209,9 +208,8 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { def testNewStrictQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = { withQuotaManager { quotaManager => quotaManager.updateQuota( - Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), - Some(ClientId), + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) val quota = quotaManager.newStrictQuotaFor(buildSession(User), ClientId) @@ -232,9 +230,8 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { def testNewPermissiveQuotaForReturnsStrictQuotaWhenQuotaIsEnabled(): Unit = { withQuotaManager { quotaManager => quotaManager.updateQuota( - Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), - Some(ClientId), + Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) val quota = quotaManager.newPermissiveQuotaFor(buildSession(User), ClientId) From 72fc80b4404afd8d4d029c1eba4dc315b7f1538c Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 10:51:00 +0800 Subject: [PATCH 04/20] update the method --- .../kafka/server/ClientQuotaManagerTest.scala | 112 ++++++++++-------- 1 file changed, 62 insertions(+), 50 deletions(-) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 67d1dae6f82d6..a16fea047fd0a 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -16,13 +16,14 @@ */ package kafka.server +import kafka.server.ClientQuotaManager.BaseUserEntity + import java.net.InetAddress import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.server.config.ClientQuotaManagerConfig import org.apache.kafka.network.Session -import org.apache.kafka.server.quota.QuotaType +import org.apache.kafka.server.quota.{ClientQuotaEntity, QuotaType} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -35,13 +36,13 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( - client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + client1.configUser, + client1.sanitizedClientId, Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( - client2.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client2.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + client2.configUser, + client2.sanitizedClientId, Some(new Quota(4000, true)) ) @@ -59,8 +60,8 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 2: Change quota again. The quota should be updated within KafkaMetrics as well since the sensor was created. // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( - client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + client1.configUser, + client1.sanitizedClientId, Some(new Quota(3000, true)) ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") @@ -70,8 +71,8 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( - client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + client1.configUser, + client1.sanitizedClientId, Some(new Quota(500, true)) ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") @@ -81,13 +82,13 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( - client1.configUser.map(s => ClientQuotaManager.UserEntity(s)), - client1.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + client1.configUser, + client1.sanitizedClientId, None ) clientQuotaManager.updateQuota( - defaultConfigClient.configUser.map(s => ClientQuotaManager.UserEntity(s)), - defaultConfigClient.sanitizedConfigClientId.map(s => ClientQuotaManager.ClientIdEntity(s)), + defaultConfigClient.configUser, + defaultConfigClient.sanitizedClientId, Some(new Quota(4000, true)) ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") @@ -106,10 +107,10 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testClientIdQuotaParsing(): Unit = { - val client1 = UserClient("ANONYMOUS", "p1", None, Some("p1")) - val client2 = UserClient("ANONYMOUS", "p2", None, Some("p2")) + val client1 = UserClient("ANONYMOUS", "p1", None, Some(ClientQuotaManager.ClientIdEntity("p1"))) + val client2 = UserClient("ANONYMOUS", "p2", None, Some(ClientQuotaManager.ClientIdEntity("p2"))) val randomClient = UserClient("ANONYMOUS", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultString)) + val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultClientIdEntity)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -119,10 +120,10 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserQuotaParsing(): Unit = { - val client1 = UserClient("User1", "p1", Some("User1"), None) - val client2 = UserClient("User2", "p2", Some("User2"), None) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), None) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), None) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultUserEntity), None) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -133,10 +134,15 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserClientIdQuotaParsing(): Unit = { - val client1 = UserClient("User1", "p1", Some("User1"), Some("p1")) - val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some(ClientQuotaManager.ClientIdEntity("p1"))) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some(ClientQuotaManager.ClientIdEntity("p2"))) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) + val defaultConfigClient = UserClient( + "", + "", + Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultClientIdEntity) + ) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -146,10 +152,10 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserQuotaParsingWithDefaultClientIdQuota(): Unit = { - val client1 = UserClient("User1", "p1", Some("User1"), None) - val client2 = UserClient("User2", "p2", Some("User2"), None) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), None) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), None) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultUserEntity), None) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -158,10 +164,15 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserClientQuotaParsingIdWithDefaultClientIdQuota(): Unit = { - val client1 = UserClient("User1", "p1", Some("User1"), Some("p1")) - val client2 = UserClient("User2", "p2", Some("User2"), Some("p2")) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some(ClientQuotaManager.ClientIdEntity("p1"))) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some(ClientQuotaManager.ClientIdEntity("p2"))) val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) + val defaultConfigClient = UserClient( + "", + "", + Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultClientIdEntity) + ) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -193,7 +204,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Set default quota config clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), None, Some(new Quota(10, true)) ) @@ -215,7 +226,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Set default quota config clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), None, Some(new Quota(10, true)) ) @@ -223,7 +234,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, back to no quotas clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), None, None ) @@ -294,18 +305,18 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), None, Some(new Quota(1000, true)) ) clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(3000, true)) ) clientQuotaManager.updateQuota( @@ -330,7 +341,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( @@ -359,8 +370,8 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, revert to default clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultClientIdEntity), None ) checkQuota(clientQuotaManager, "userD", "client1", 1000, 0, expectThrottle = false) // Metrics tags changed, restart counter @@ -370,7 +381,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, revert to default clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultUserEntity), None, None ) @@ -408,7 +419,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(12000, true)) ) clientQuotaManager.updateQuota( @@ -430,7 +441,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -481,7 +492,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -506,7 +517,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -536,7 +547,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.ClientIdEntity(ClientQuotaManager.DefaultString)), + Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -553,10 +564,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } } - private case class UserClient(user: String, clientId: String, configUser: Option[String] = None, configClientId: Option[String] = None) { - // The class under test expects only sanitized client configs. We pass both the default value (which should not be - // sanitized to ensure it remains unique) and non-default values, so we need to take care in generating the sanitized - // client ID - def sanitizedConfigClientId = configClientId.map(x => if (x == ClientQuotaManager.DefaultString) ClientQuotaManager.DefaultString else Sanitizer.sanitize(x)) + private case class UserClient( + user: String, + clientId: String, + configUser: Option[BaseUserEntity] = None, + sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] = None + ) { } } From 8065ee712f878d85a4c740d8b1fa9c50a2bc9deb Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 11:10:17 +0800 Subject: [PATCH 05/20] add new test for new method --- .../metadata/ClientQuotaMetadataManager.scala | 49 ++++++++++++------- .../ClientQuotaMetadataManagerTest.scala | 42 +++++++++++++++- 2 files changed, 71 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 49994189ae267..854066d185a8d 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -19,11 +19,14 @@ package kafka.server.metadata import kafka.network.ConnectionQuotas import kafka.server.ClientQuotaManager +import kafka.server.ClientQuotaManager.BaseUserEntity import kafka.server.QuotaFactory.QuotaManagers +import kafka.server.metadata.ClientQuotaMetadataManager.transferToClientQuotaEntity import kafka.utils.Logging import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.quota.ClientQuotaEntity import org.apache.kafka.common.utils.Sanitizer +import org.apache.kafka.server.quota.ClientQuotaEntity.{ConfigEntity => ClientQuotaConfigEntity} import java.net.{InetAddress, UnknownHostException} import org.apache.kafka.image.{ClientQuotaDelta, ClientQuotasDelta} @@ -146,25 +149,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag } // Convert entity into Options with sanitized values for QuotaManagers - val (sanitizedUser, sanitizedClientId) = quotaEntity match { - case UserEntity(user) => - (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) - case DefaultUserEntity => - (Some(ClientQuotaManager.DefaultUserEntity), None) - case ClientIdEntity(clientId) => - (None, Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) - case DefaultClientIdEntity => - (None, Some(ClientQuotaManager.DefaultClientIdEntity)) - case ExplicitUserExplicitClientIdEntity(user, clientId) => - (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) - case ExplicitUserDefaultClientIdEntity(user) => - (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultClientIdEntity)) - case DefaultUserExplicitClientIdEntity(clientId) => - (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) - case DefaultUserDefaultClientIdEntity => - (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) - case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") - } + val (sanitizedUser, sanitizedClientId) = transferToClientQuotaEntity(quotaEntity) val quotaValue = newValue.map(new Quota(_, true)) try { @@ -178,3 +163,29 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag } } } + +object ClientQuotaMetadataManager { + + def transferToClientQuotaEntity(quotaEntity: QuotaEntity): (Option[BaseUserEntity], Option[ClientQuotaConfigEntity]) = { + val (sanitizedUser, sanitizedClientId) = quotaEntity match { + case UserEntity(user) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) + case DefaultUserEntity => + (Some(ClientQuotaManager.DefaultUserEntity), None) + case ClientIdEntity(clientId) => + (None, Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case DefaultClientIdEntity => + (None, Some(ClientQuotaManager.DefaultClientIdEntity)) + case ExplicitUserExplicitClientIdEntity(user, clientId) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case ExplicitUserDefaultClientIdEntity(user) => + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultClientIdEntity)) + case DefaultUserExplicitClientIdEntity(clientId) => + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + case DefaultUserDefaultClientIdEntity => + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) + case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") + } + (sanitizedUser, sanitizedClientId) + } +} \ No newline at end of file diff --git a/core/src/test/scala/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala b/core/src/test/scala/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala index 03d8d30cea056..9d36cae25c239 100644 --- a/core/src/test/scala/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala +++ b/core/src/test/scala/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala @@ -16,8 +16,9 @@ */ package kafka.server.metadata +import kafka.server.ClientQuotaManager import org.apache.kafka.image.ClientQuotaDelta -import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertThrows} +import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertThrows} import org.junit.jupiter.api.Test import org.junit.jupiter.api.function.Executable @@ -33,4 +34,43 @@ class ClientQuotaMetadataManagerTest { assertDoesNotThrow { new Executable { def execute(): Unit = manager.handleIpQuota(IpEntity("192.168.1.1"), new ClientQuotaDelta(null)) } } assertDoesNotThrow { new Executable { def execute(): Unit = manager.handleIpQuota(IpEntity("2001:db8::1"), new ClientQuotaDelta(null)) } } } + + @Test + def testTransferToClientQuotaEntity(): Unit = { + + assertThrows(classOf[IllegalStateException],() => ClientQuotaMetadataManager.transferToClientQuotaEntity(IpEntity("a"))) + assertThrows(classOf[IllegalStateException],() => ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultIpEntity)) + assertEquals( + (Some(ClientQuotaManager.UserEntity("user")), None), + ClientQuotaMetadataManager.transferToClientQuotaEntity(UserEntity("user")) + ) + assertEquals( + (Some(ClientQuotaManager.DefaultUserEntity), None), + ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserEntity) + ) + assertEquals( + (None, Some(ClientQuotaManager.ClientIdEntity("client"))), + ClientQuotaMetadataManager.transferToClientQuotaEntity(ClientIdEntity("client")) + ) + assertEquals( + (None, Some(ClientQuotaManager.DefaultClientIdEntity)), + ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultClientIdEntity) + ) + assertEquals( + (Some(ClientQuotaManager.UserEntity("user")), Some(ClientQuotaManager.ClientIdEntity("client"))), + ClientQuotaMetadataManager.transferToClientQuotaEntity(ExplicitUserExplicitClientIdEntity("user", "client")) + ) + assertEquals( + (Some(ClientQuotaManager.UserEntity("user")), Some(ClientQuotaManager.DefaultClientIdEntity)), + ClientQuotaMetadataManager.transferToClientQuotaEntity(ExplicitUserDefaultClientIdEntity("user")) + ) + assertEquals( + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity("client"))), + ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserExplicitClientIdEntity("client")) + ) + assertEquals( + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)), + ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserDefaultClientIdEntity) + ) + } } \ No newline at end of file From ff19654b895cea1d78dba125f00534d9d4575a58 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 13:54:32 +0800 Subject: [PATCH 06/20] wip --- core/src/main/scala/kafka/server/ClientQuotaManager.scala | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index d588c4b9e083c..20b45faf41ad7 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -452,14 +452,18 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, private def getOrDefaultClient( sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] ): Option[ClientQuotaEntity.ConfigEntity] = { - if (sanitizedClientId.nonEmpty && sanitizedClientId.get.name() == DefaultString) + if (sanitizedClientId.isEmpty) + None + else if (sanitizedClientId.get.name() == DefaultString) Some(DefaultClientIdEntity) else sanitizedClientId } private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { - if (sanitizedUser.nonEmpty && sanitizedUser.get.name() == DefaultString) + if (sanitizedUser.isEmpty) + None + else if (sanitizedUser.get.name() == DefaultString) Some(DefaultUserEntity) else sanitizedUser From 821acd05ca87a5a55fd0861a26d0da3273419f28 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 14:43:40 +0800 Subject: [PATCH 07/20] wip --- .../kafka/server/ClientQuotaManager.scala | 13 ++- .../metadata/ClientQuotaMetadataManager.scala | 1 + .../DynamicBrokerReconfigurationTest.scala | 1 + .../kafka/server/ClientQuotaManagerTest.scala | 101 ++++++++++++------ .../ClientRequestQuotaManagerTest.scala | 1 + .../ControllerMutationQuotaManagerTest.scala | 3 + 6 files changed, 83 insertions(+), 37 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 20b45faf41ad7..38b4a751e81b9 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -404,11 +404,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * for any of these levels. * * @param sanitizedUser user to override if quota applies to or + * @param clientId client to override if quota applies to or * @param sanitizedClientId sanitized client ID to override if quota applies to or * @param quota custom quota to apply or None if quota override is being removed */ def updateQuota( - sanitizedUser: Option[BaseUserEntity], + sanitizedUser: Option[BaseUserEntity], + clientId: Option[String], sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], quota: Option[Quota] ): Unit = { @@ -422,7 +424,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, lock.writeLock().lock() try { val userEntity = getOrDefaultUser(sanitizedUser) - val clientIdEntity = getOrDefaultClient(sanitizedClientId) + val clientIdEntity = getOrDefaultClient(clientId, sanitizedClientId) val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) @@ -450,14 +452,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } private def getOrDefaultClient( + clientId: Option[String], sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] ): Option[ClientQuotaEntity.ConfigEntity] = { if (sanitizedClientId.isEmpty) - None - else if (sanitizedClientId.get.name() == DefaultString) + None + else if (sanitizedClientId.nonEmpty && sanitizedClientId.get.name() == DefaultString) Some(DefaultClientIdEntity) else - sanitizedClientId + Some(ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))) } private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 854066d185a8d..5d5417582eccf 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -155,6 +155,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag try { manager.updateQuota( sanitizedUser = sanitizedUser, + clientId = sanitizedClientId.map(s => Sanitizer.desanitize(s.name())), sanitizedClientId = sanitizedClientId, quota = quotaValue ) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index f4de50d7cd8a3..e8353718865c0 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -936,6 +936,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup servers.foreach { server => server.quotaManagers.produce.updateQuota( None, + Some(clientId), Some(ClientQuotaManager.ClientIdEntity(clientId)), Some(Quota.upperBound(10000000)) ) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index a16fea047fd0a..6933c08359f44 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -21,9 +21,10 @@ import kafka.server.ClientQuotaManager.BaseUserEntity import java.net.InetAddress import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.server.config.ClientQuotaManagerConfig import org.apache.kafka.network.Session -import org.apache.kafka.server.quota.{ClientQuotaEntity, QuotaType} +import org.apache.kafka.server.quota.QuotaType import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Test @@ -36,13 +37,15 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( - client1.configUser, - client1.sanitizedClientId, + client1.configUser, + client1.configClientId, + client1.sanitizedConfigClientId, Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( - client2.configUser, - client2.sanitizedClientId, + client2.configUser, + client2.configClientId, + client2.sanitizedConfigClientId, Some(new Quota(4000, true)) ) @@ -60,8 +63,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 2: Change quota again. The quota should be updated within KafkaMetrics as well since the sensor was created. // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( - client1.configUser, - client1.sanitizedClientId, + client1.configUser, + client1.configClientId, + client1.sanitizedConfigClientId, Some(new Quota(3000, true)) ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") @@ -71,8 +75,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( - client1.configUser, - client1.sanitizedClientId, + client1.configUser, + client1.configClientId, + client1.sanitizedConfigClientId, Some(new Quota(500, true)) ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") @@ -82,13 +87,15 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( - client1.configUser, - client1.sanitizedClientId, + client1.configUser, + client1.configClientId, + client1.sanitizedConfigClientId, None ) clientQuotaManager.updateQuota( - defaultConfigClient.configUser, - defaultConfigClient.sanitizedClientId, + defaultConfigClient.configUser, + defaultConfigClient.configClientId, + defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true)) ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") @@ -107,10 +114,10 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testClientIdQuotaParsing(): Unit = { - val client1 = UserClient("ANONYMOUS", "p1", None, Some(ClientQuotaManager.ClientIdEntity("p1"))) - val client2 = UserClient("ANONYMOUS", "p2", None, Some(ClientQuotaManager.ClientIdEntity("p2"))) + val client1 = UserClient("ANONYMOUS", "p1", None, Some("p1")) + val client2 = UserClient("ANONYMOUS", "p2", None, Some("p2")) val randomClient = UserClient("ANONYMOUS", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultClientIdEntity)) + val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultClientIdEntity.name)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -134,14 +141,14 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserClientIdQuotaParsing(): Unit = { - val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some(ClientQuotaManager.ClientIdEntity("p1"))) - val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some(ClientQuotaManager.ClientIdEntity("p2"))) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some("p1")) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some("p2")) val randomClient = UserClient("RandomUser", "random-client-id", None, None) val defaultConfigClient = UserClient( "", "", Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultClientIdEntity) + Some(ClientQuotaManager.DefaultClientIdEntity.name) ) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) @@ -164,14 +171,14 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { */ @Test def testUserClientQuotaParsingIdWithDefaultClientIdQuota(): Unit = { - val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some(ClientQuotaManager.ClientIdEntity("p1"))) - val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some(ClientQuotaManager.ClientIdEntity("p2"))) + val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some("p1")) + val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some("p2")) val randomClient = UserClient("RandomUser", "random-client-id", None, None) val defaultConfigClient = UserClient( "", "", Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultClientIdEntity) + Some(ClientQuotaManager.DefaultClientIdEntity.name) ) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -206,6 +213,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, + None, Some(new Quota(10, true)) ) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) @@ -228,6 +236,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, + None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -236,6 +245,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, + None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -255,6 +265,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, + None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -263,6 +274,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, + None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -280,7 +292,8 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { // Set quota config clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10, true)) ) @@ -289,6 +302,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove quota config, back to no quotas clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), None ) @@ -307,50 +321,60 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, + None, Some(new Quota(1000, true)) ) clientQuotaManager.updateQuota( None, + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(3000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, + None, Some(new Quota(4000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(5000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), None, + None, Some(new Quota(6000, true)) ) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userB")), + Some(ClientQuotaManager.UserEntity("userB")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(7000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userC")), None, + None, Some(new Quota(10000, true)) ) clientQuotaManager.updateQuota( - None, + None, + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(9000, true)) ) @@ -371,6 +395,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, revert to default clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), None ) @@ -383,6 +408,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, + None, None ) checkQuota(clientQuotaManager, "userF", "client4", 2000, 0, expectThrottle = false) // Default quota shared across client-id of all users @@ -394,10 +420,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, + None, Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10000, true)) ) @@ -406,24 +434,28 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client1", 10000, 0, expectThrottle = false) checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, expectThrottle = true) clientQuotaManager.updateQuota( - Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.UserEntity("userA")), + Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), None ) checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, expectThrottle = true) // Throttled due to shared user quota clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some("client6"), Some(ClientQuotaManager.ClientIdEntity("client6")), Some(new Quota(11000, true)) ) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(12000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), + Some("client6"), Some(ClientQuotaManager.ClientIdEntity("client6")), None ) @@ -441,6 +473,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -492,6 +525,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -517,6 +551,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -547,6 +582,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, + Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -564,11 +600,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } } - private case class UserClient( - user: String, - clientId: String, - configUser: Option[BaseUserEntity] = None, - sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] = None - ) { + private case class UserClient(user: String, clientId: String, configUser: Option[BaseUserEntity] = None, configClientId: Option[String] = None) { + // The class under test expects only sanitized client configs. We pass both the default value (which should not be + // sanitized to ensure it remains unique) and non-default values, so we need to take care in generating the sanitized + // client ID + def sanitizedConfigClientId = + configClientId.map(x => if (x == ClientQuotaManager.DefaultString) ClientQuotaManager.DefaultString else Sanitizer.sanitize(x)) + .map(ClientQuotaManager.ClientIdEntity) } } diff --git a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala index 368280d235453..9a8b61f65686c 100644 --- a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala @@ -32,6 +32,7 @@ class ClientRequestQuotaManagerTest extends BaseClientQuotaManagerTest { val clientRequestQuotaManager = new ClientRequestQuotaManager(config, metrics, time, "", Optional.empty()) clientRequestQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("ANONYMOUS")), + Some("test-client"), Some(ClientQuotaManager.ClientIdEntity("test-client")), Some(Quota.upperBound(1)) ) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala index 2a7cfe35a85b0..39d9e09fcfc4c 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala @@ -147,6 +147,7 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) @@ -209,6 +210,7 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) @@ -231,6 +233,7 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), + Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) From 2d165651c30c220486beace65d9dab23f6363da5 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 14:52:12 +0800 Subject: [PATCH 08/20] fix the error tests --- .../kafka/server/ClientQuotaManager.scala | 11 +++--- .../metadata/ClientQuotaMetadataManager.scala | 1 - .../DynamicBrokerReconfigurationTest.scala | 1 - .../kafka/server/ClientQuotaManagerTest.scala | 35 ------------------- .../ClientRequestQuotaManagerTest.scala | 1 - .../ControllerMutationQuotaManagerTest.scala | 3 -- 6 files changed, 5 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 38b4a751e81b9..d883cd83a33b0 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -404,13 +404,11 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * for any of these levels. * * @param sanitizedUser user to override if quota applies to or - * @param clientId client to override if quota applies to or * @param sanitizedClientId sanitized client ID to override if quota applies to or * @param quota custom quota to apply or None if quota override is being removed */ def updateQuota( sanitizedUser: Option[BaseUserEntity], - clientId: Option[String], sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], quota: Option[Quota] ): Unit = { @@ -424,7 +422,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, lock.writeLock().lock() try { val userEntity = getOrDefaultUser(sanitizedUser) - val clientIdEntity = getOrDefaultClient(clientId, sanitizedClientId) + val clientIdEntity = getOrDefaultClient(sanitizedClientId) val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) @@ -452,15 +450,16 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } private def getOrDefaultClient( - clientId: Option[String], sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] ): Option[ClientQuotaEntity.ConfigEntity] = { if (sanitizedClientId.isEmpty) None - else if (sanitizedClientId.nonEmpty && sanitizedClientId.get.name() == DefaultString) + else if (sanitizedClientId.get.name() == DefaultString) Some(DefaultClientIdEntity) - else + else { + val clientId = sanitizedClientId.map(s => Sanitizer.desanitize(s.name())) Some(ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))) + } } private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 5d5417582eccf..854066d185a8d 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -155,7 +155,6 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag try { manager.updateQuota( sanitizedUser = sanitizedUser, - clientId = sanitizedClientId.map(s => Sanitizer.desanitize(s.name())), sanitizedClientId = sanitizedClientId, quota = quotaValue ) diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala index e8353718865c0..f4de50d7cd8a3 100644 --- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala +++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala @@ -936,7 +936,6 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup servers.foreach { server => server.quotaManagers.produce.updateQuota( None, - Some(clientId), Some(ClientQuotaManager.ClientIdEntity(clientId)), Some(Quota.upperBound(10000000)) ) diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 6933c08359f44..f1785ea759518 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -38,13 +38,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( client1.configUser, - client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( client2.configUser, - client2.configClientId, client2.sanitizedConfigClientId, Some(new Quota(4000, true)) ) @@ -64,7 +62,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( client1.configUser, - client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(3000, true)) ) @@ -76,7 +73,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( client1.configUser, - client1.configClientId, client1.sanitizedConfigClientId, Some(new Quota(500, true)) ) @@ -88,13 +84,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( client1.configUser, - client1.configClientId, client1.sanitizedConfigClientId, None ) clientQuotaManager.updateQuota( defaultConfigClient.configUser, - defaultConfigClient.configClientId, defaultConfigClient.sanitizedConfigClientId, Some(new Quota(4000, true)) ) @@ -213,7 +207,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, - None, Some(new Quota(10, true)) ) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) @@ -236,7 +229,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, - None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -245,7 +237,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, - None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -265,7 +256,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(10, true)) ) checkQuota(clientQuotaManager, "userA", "client1", 10, 1000, expectThrottle = true) @@ -274,7 +264,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, None ) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) @@ -293,7 +282,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Set quota config clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10, true)) ) @@ -302,7 +290,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove quota config, back to no quotas clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), None ) @@ -321,60 +308,50 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, - None, Some(new Quota(1000, true)) ) clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(3000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(4000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(5000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), None, - None, Some(new Quota(6000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(7000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userB")), - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userC")), None, - None, Some(new Quota(10000, true)) ) clientQuotaManager.updateQuota( None, - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(9000, true)) ) @@ -395,7 +372,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Remove default quota config, revert to default clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), None ) @@ -408,7 +384,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.DefaultUserEntity), None, - None, None ) checkQuota(clientQuotaManager, "userF", "client4", 2000, 0, expectThrottle = false) // Default quota shared across client-id of all users @@ -420,12 +395,10 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), None, - None, Some(new Quota(8000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), Some(new Quota(10000, true)) ) @@ -435,27 +408,23 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client1", 10000, 6000, expectThrottle = true) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client1"), Some(ClientQuotaManager.ClientIdEntity("client1")), None ) checkQuota(clientQuotaManager, "userA", "client6", 8000, 0, expectThrottle = true) // Throttled due to shared user quota clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client6"), Some(ClientQuotaManager.ClientIdEntity("client6")), Some(new Quota(11000, true)) ) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(12000, true)) ) clientQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("userA")), - Some("client6"), Some(ClientQuotaManager.ClientIdEntity("client6")), None ) @@ -473,7 +442,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -525,7 +493,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -551,7 +518,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) @@ -582,7 +548,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { try { clientQuotaManager.updateQuota( None, - Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultClientIdEntity), Some(new Quota(500, true)) ) diff --git a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala index 9a8b61f65686c..368280d235453 100644 --- a/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientRequestQuotaManagerTest.scala @@ -32,7 +32,6 @@ class ClientRequestQuotaManagerTest extends BaseClientQuotaManagerTest { val clientRequestQuotaManager = new ClientRequestQuotaManager(config, metrics, time, "", Optional.empty()) clientRequestQuotaManager.updateQuota( Some(ClientQuotaManager.UserEntity("ANONYMOUS")), - Some("test-client"), Some(ClientQuotaManager.ClientIdEntity("test-client")), Some(Quota.upperBound(1)) ) diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala index 39d9e09fcfc4c..2a7cfe35a85b0 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaManagerTest.scala @@ -147,7 +147,6 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) @@ -210,7 +209,6 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) @@ -233,7 +231,6 @@ class ControllerMutationQuotaManagerTest extends BaseClientQuotaManagerTest { withQuotaManager { quotaManager => quotaManager.updateQuota( Some(User).map(s => ClientQuotaManager.UserEntity(s)), - Some(ClientId), Some(ClientQuotaManager.ClientIdEntity(ClientId)), Some(Quota.upperBound(10)) ) From fe8b6cae254cd2322bebfbf691886c7e8b564087 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 14:58:11 +0800 Subject: [PATCH 09/20] update the comment --- core/src/main/scala/kafka/server/ClientQuotaManager.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index d883cd83a33b0..827e334a78b62 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -472,9 +472,8 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } /** - * Updates metrics configs. This is invoked when quota configs are updated in ZooKeeper - * or when partitions leaders change and custom callbacks that implement partition-based quotas - * have updated quotas. + * Updates metrics configs. This is invoked when quota configs are updated when partitions leaders change + * and custom callbacks that implement partition-based quotas have updated quotas. * * @param updatedQuotaEntity If set to one entity and quotas have only been enabled at one * level, then an optimized update is performed with a single metric update. If None is provided, From b81ae527c911886ca08395b56333d8d9d28370cf Mon Sep 17 00:00:00 2001 From: m1a2st Date: Fri, 31 Jan 2025 15:05:03 +0800 Subject: [PATCH 10/20] update the comment --- .../main/scala/kafka/server/DynamicBrokerConfig.scala | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index d210ca06f93fd..06e7be6111c64 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -48,13 +48,6 @@ import scala.collection._ import scala.jdk.CollectionConverters._ /** - * Dynamic broker configurations are stored in ZooKeeper and may be defined at two levels: - *
    - *
  • Per-broker configs persisted at /configs/brokers/{brokerId}: These can be described/altered - * using AdminClient using the resource name brokerId.
  • - *
  • Cluster-wide defaults persisted at /configs/brokers/<default>: These can be described/altered - * using AdminClient using an empty resource name.
  • - *
* The order of precedence for broker configs is: *
    *
  1. STATIC_BROKER_CONFIG: properties that broker is started up with, typically from server.properties file
  2. @@ -391,7 +384,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging error(s"$errorMessage: $invalidPropNames") } } - removeInvalidProps(nonDynamicConfigs(props), "Non-dynamic configs configured in ZooKeeper will be ignored") + removeInvalidProps(nonDynamicConfigs(props), "Non-dynamic configs will be ignored") removeInvalidProps(securityConfigsWithoutListenerPrefix(props), "Security configs can be dynamically updated only using listener prefix, base configs will be ignored") if (!perBrokerConfig) From 4ca7f31bfdcd52d131dd98e13534747cff78b16f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 1 Feb 2025 11:54:50 +0800 Subject: [PATCH 11/20] update the userEntity logic --- .../kafka/server/ClientQuotaManager.scala | 19 ++++--------------- .../metadata/ClientQuotaMetadataManager.scala | 8 ++++---- 2 files changed, 8 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 827e334a78b62..dcedf70d0ce0f 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -403,15 +403,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * Overrides quotas for , or or the dynamic defaults * for any of these levels. * - * @param sanitizedUser user to override if quota applies to or + * @param userEntity user to override if quota applies to or * @param sanitizedClientId sanitized client ID to override if quota applies to or * @param quota custom quota to apply or None if quota override is being removed */ def updateQuota( - sanitizedUser: Option[BaseUserEntity], - sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], + userEntity: Option[BaseUserEntity], + sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], quota: Option[Quota] - ): Unit = { + ): Unit = { /* * Acquire the write lock to apply changes in the quota objects. * This method changes the quota in the overriddenQuota map and applies the update on the actual KafkaMetric object (if it exists). @@ -421,9 +421,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val userEntity = getOrDefaultUser(sanitizedUser) val clientIdEntity = getOrDefaultClient(sanitizedClientId) - val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) if (userEntity.nonEmpty) { @@ -462,15 +460,6 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } } - private def getOrDefaultUser(sanitizedUser: Option[BaseUserEntity]): Option[BaseUserEntity] = { - if (sanitizedUser.isEmpty) - None - else if (sanitizedUser.get.name() == DefaultString) - Some(DefaultUserEntity) - else - sanitizedUser - } - /** * Updates metrics configs. This is invoked when quota configs are updated when partitions leaders change * and custom callbacks that implement partition-based quotas have updated quotas. diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 854066d185a8d..483f8b9fa7409 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -149,12 +149,12 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag } // Convert entity into Options with sanitized values for QuotaManagers - val (sanitizedUser, sanitizedClientId) = transferToClientQuotaEntity(quotaEntity) + val (userEntity, sanitizedClientId) = transferToClientQuotaEntity(quotaEntity) val quotaValue = newValue.map(new Quota(_, true)) try { manager.updateQuota( - sanitizedUser = sanitizedUser, + userEntity = userEntity, sanitizedClientId = sanitizedClientId, quota = quotaValue ) @@ -167,7 +167,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag object ClientQuotaMetadataManager { def transferToClientQuotaEntity(quotaEntity: QuotaEntity): (Option[BaseUserEntity], Option[ClientQuotaConfigEntity]) = { - val (sanitizedUser, sanitizedClientId) = quotaEntity match { + val (userEntity, sanitizedClientId) = quotaEntity match { case UserEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) case DefaultUserEntity => @@ -186,6 +186,6 @@ object ClientQuotaMetadataManager { (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } - (sanitizedUser, sanitizedClientId) + (userEntity, sanitizedClientId) } } \ No newline at end of file From a6a52e0a6614591bcac6d042b24a56e89b1c08f3 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 1 Feb 2025 12:12:12 +0800 Subject: [PATCH 12/20] update the clientIdEntity logic --- .../kafka/server/ClientQuotaManager.scala | 21 +++----- .../admin/ConfigCommandIntegrationTest.java | 26 ++++++++++ .../kafka/server/ClientQuotaManagerTest.scala | 49 ------------------- 3 files changed, 33 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index dcedf70d0ce0f..c6334da73ae91 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -421,7 +421,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val clientIdEntity = getOrDefaultClient(sanitizedClientId) + val clientIdEntity = sanitizedClientId match { + case Some(sanitizedClientId: ClientIdEntity) => + val clientId = Some(sanitizedClientId).map(s => Sanitizer.desanitize(s.name)) + Some(ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))) + case _ => sanitizedClientId + } + val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) if (userEntity.nonEmpty) { @@ -447,19 +453,6 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, } } - private def getOrDefaultClient( - sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity] - ): Option[ClientQuotaEntity.ConfigEntity] = { - if (sanitizedClientId.isEmpty) - None - else if (sanitizedClientId.get.name() == DefaultString) - Some(DefaultClientIdEntity) - else { - val clientId = sanitizedClientId.map(s => Sanitizer.desanitize(s.name())) - Some(ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))) - } - } - /** * Updates metrics configs. This is invoked when quota configs are updated when partitions leaders change * and custom callbacks that implement partition-based quotas have updated quotas. diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 9aa7c9da2fac3..404bceb7607bf 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -21,10 +21,14 @@ import org.apache.kafka.clients.admin.AlterConfigsOptions; import org.apache.kafka.clients.admin.AlterConfigsResult; import org.apache.kafka.clients.admin.ConfigEntry; +import org.apache.kafka.clients.admin.DescribeClientQuotasResult; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.quota.ClientQuotaAlteration; +import org.apache.kafka.common.quota.ClientQuotaEntity; +import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; @@ -37,6 +41,7 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -91,6 +96,27 @@ public ConfigCommandIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } + @ClusterTest(types = {Type.KRAFT}) + public void test() throws ExecutionException, InterruptedException { + try (Admin admin = cluster.admin()) { + Map map = new HashMap<>(); + map.put("user", ""); + ClientQuotaEntity entity = new ClientQuotaEntity(map); + ClientQuotaFilter filter = ClientQuotaFilter.all(); + + ArrayList quotas = new ArrayList<>(); + quotas.add(new ClientQuotaAlteration(entity, Collections.singleton(new ClientQuotaAlteration.Op("producer_byte_rate", 1000.0)))); + admin.alterClientQuotas(quotas); + Thread.sleep(5000); + DescribeClientQuotasResult describeClientQuotasResult = admin.describeClientQuotas(filter); + describeClientQuotasResult.entities().get().forEach( + (entity1, quotas1) -> quotas1.forEach( + (quotaType, value) -> System.out.println(entity1 + " " + quotaType + " " + value) + ) + ); + } + } + @ClusterTest public void testExitWithNonZeroStatusOnUpdatingUnallowedConfig() { assertNonZeroStatusExit(Stream.concat(quorumArgs(), Stream.of( diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index f1785ea759518..74102f86892d8 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -102,19 +102,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } } - /** - * Tests parsing for quotas. - * Quota overrides persisted in ZooKeeper in /config/clients/, default persisted in /config/clients/ - */ - @Test - def testClientIdQuotaParsing(): Unit = { - val client1 = UserClient("ANONYMOUS", "p1", None, Some("p1")) - val client2 = UserClient("ANONYMOUS", "p2", None, Some("p2")) - val randomClient = UserClient("ANONYMOUS", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultClientIdEntity.name)) - testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) - } - /** * Tests parsing for quotas. * Quota overrides persisted in ZooKeeper in /config/users/, default persisted in /config/users/ @@ -129,25 +116,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } - /** - * Tests parsing for quotas. - * Quotas persisted in ZooKeeper in /config/users//clients/, default in /config/users//clients/ - */ - @Test - def testUserClientIdQuotaParsing(): Unit = { - val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some("p1")) - val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some("p2")) - val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient( - "", - "", - Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultClientIdEntity.name) - ) - val config = new ClientQuotaManagerConfig() - testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) - } - /** * Tests parsing for quotas when client-id default quota properties are set. */ @@ -160,23 +128,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } - /** - * Tests parsing for quotas when client-id default quota properties are set. - */ - @Test - def testUserClientQuotaParsingIdWithDefaultClientIdQuota(): Unit = { - val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), Some("p1")) - val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), Some("p2")) - val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient( - "", - "", - Some(ClientQuotaManager.DefaultUserEntity), - Some(ClientQuotaManager.DefaultClientIdEntity.name) - ) - testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) - } - private def checkQuota(quotaManager: ClientQuotaManager, user: String, clientId: String, expectedBound: Long, value: Int, expectThrottle: Boolean): Unit = { assertEquals(expectedBound.toDouble, quotaManager.quota(user, clientId).bound, 0.0) val session = new Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, user), InetAddress.getLocalHost) From 44274d4776e2377a0e7b26d812e033e77182e3da Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 1 Feb 2025 12:21:59 +0800 Subject: [PATCH 13/20] update the clientIdEntity logic --- .../kafka/server/ClientQuotaManager.scala | 4 +-- .../admin/ConfigCommandIntegrationTest.java | 26 ------------------- .../kafka/server/ClientQuotaManagerTest.scala | 5 ++-- .../server/ClientQuotasRequestTest.scala | 14 ---------- 4 files changed, 3 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index c6334da73ae91..e498283b96267 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -422,9 +422,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, lock.writeLock().lock() try { val clientIdEntity = sanitizedClientId match { - case Some(sanitizedClientId: ClientIdEntity) => - val clientId = Some(sanitizedClientId).map(s => Sanitizer.desanitize(s.name)) - Some(ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))) + case Some(client: ClientIdEntity) => Some(ClientIdEntity(Sanitizer.desanitize(client.name))) case _ => sanitizedClientId } diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 404bceb7607bf..9aa7c9da2fac3 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -21,14 +21,10 @@ import org.apache.kafka.clients.admin.AlterConfigsOptions; import org.apache.kafka.clients.admin.AlterConfigsResult; import org.apache.kafka.clients.admin.ConfigEntry; -import org.apache.kafka.clients.admin.DescribeClientQuotasResult; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; -import org.apache.kafka.common.quota.ClientQuotaAlteration; -import org.apache.kafka.common.quota.ClientQuotaEntity; -import org.apache.kafka.common.quota.ClientQuotaFilter; import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; import org.apache.kafka.common.test.api.ClusterTest; @@ -41,7 +37,6 @@ import java.io.ByteArrayOutputStream; import java.io.PrintStream; -import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -96,27 +91,6 @@ public ConfigCommandIntegrationTest(ClusterInstance cluster) { this.cluster = cluster; } - @ClusterTest(types = {Type.KRAFT}) - public void test() throws ExecutionException, InterruptedException { - try (Admin admin = cluster.admin()) { - Map map = new HashMap<>(); - map.put("user", ""); - ClientQuotaEntity entity = new ClientQuotaEntity(map); - ClientQuotaFilter filter = ClientQuotaFilter.all(); - - ArrayList quotas = new ArrayList<>(); - quotas.add(new ClientQuotaAlteration(entity, Collections.singleton(new ClientQuotaAlteration.Op("producer_byte_rate", 1000.0)))); - admin.alterClientQuotas(quotas); - Thread.sleep(5000); - DescribeClientQuotasResult describeClientQuotasResult = admin.describeClientQuotas(filter); - describeClientQuotasResult.entities().get().forEach( - (entity1, quotas1) -> quotas1.forEach( - (quotaType, value) -> System.out.println(entity1 + " " + quotaType + " " + value) - ) - ); - } - } - @ClusterTest public void testExitWithNonZeroStatusOnUpdatingUnallowedConfig() { assertNonZeroStatusExit(Stream.concat(quorumArgs(), Stream.of( diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 74102f86892d8..e42e68ae16ad4 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -520,8 +520,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // The class under test expects only sanitized client configs. We pass both the default value (which should not be // sanitized to ensure it remains unique) and non-default values, so we need to take care in generating the sanitized // client ID - def sanitizedConfigClientId = - configClientId.map(x => if (x == ClientQuotaManager.DefaultString) ClientQuotaManager.DefaultString else Sanitizer.sanitize(x)) - .map(ClientQuotaManager.ClientIdEntity) + def sanitizedConfigClientId = configClientId.map(x => Sanitizer.sanitize(x)) + .map(ClientQuotaManager.ClientIdEntity) } } diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index 9dadefd72ab39..8c30f749427fc 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -517,20 +517,6 @@ class ClientQuotasRequestTest(cluster: ClusterInstance) { )) } - @ClusterTest - def testClientQuotasWithDefaultName(): Unit = { - // An entity using the name associated with the default entity name. The entity's name should be sanitized so - // that it does not conflict with the default entity name. - val entity = new ClientQuotaEntity(Map(ClientQuotaEntity.CLIENT_ID -> ClientQuotaManager.DefaultString).asJava) - alterEntityQuotas(entity, Map(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> Some(20000.0)), validateOnly = false) - verifyDescribeEntityQuotas(entity, Map(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG -> 20000.0)) - - // This should not match. - val result = describeClientQuotas( - ClientQuotaFilter.containsOnly(List(ClientQuotaFilterComponent.ofDefaultEntity(ClientQuotaEntity.CLIENT_ID)).asJava)) - assert(result.isEmpty) - } - private def verifyDescribeEntityQuotas(entity: ClientQuotaEntity, quotas: Map[String, Double]): Unit = { TestUtils.tryUntilNoAssertionError(waitTime = 5000L) { val components = entity.entries.asScala.map { case (entityType, entityName) => From e9b1e5bd8c51dc4f4eea4b2a5a713df69e515821 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sat, 1 Feb 2025 12:50:14 +0800 Subject: [PATCH 14/20] make string private --- core/src/main/scala/kafka/server/ClientQuotaManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index e498283b96267..886d097d4342a 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -55,7 +55,7 @@ object QuotaTypes { object ClientQuotaManager { // Purge sensors after 1 hour of inactivity val InactiveSensorExpirationTimeSeconds = 3600 - val DefaultString = "" + private val DefaultString = "" val DefaultClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity)) val DefaultUserQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None) val DefaultUserClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity)) From 192716617dbca7f7ad1a33a5b2db5241ff0b128b Mon Sep 17 00:00:00 2001 From: m1a2st Date: Sun, 2 Feb 2025 11:11:12 +0800 Subject: [PATCH 15/20] fix some detail --- .../kafka/server/ClientQuotaManager.scala | 12 +++--- .../metadata/ClientQuotaMetadataManager.scala | 8 ++-- .../kafka/server/ClientQuotaManagerTest.scala | 39 ++++++------------- 3 files changed, 21 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 886d097d4342a..224bd9b8b7e09 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -403,13 +403,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * Overrides quotas for , or or the dynamic defaults * for any of these levels. * - * @param userEntity user to override if quota applies to or - * @param sanitizedClientId sanitized client ID to override if quota applies to or - * @param quota custom quota to apply or None if quota override is being removed + * @param userEntity user to override if quota applies to or + * @param sanitizedClientEntity sanitized client entity to override if quota applies to or + * @param quota custom quota to apply or None if quota override is being removed */ def updateQuota( userEntity: Option[BaseUserEntity], - sanitizedClientId: Option[ClientQuotaEntity.ConfigEntity], + sanitizedClientEntity: Option[ClientQuotaEntity.ConfigEntity], quota: Option[Quota] ): Unit = { /* @@ -421,9 +421,9 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val clientIdEntity = sanitizedClientId match { + val clientIdEntity = sanitizedClientEntity match { case Some(client: ClientIdEntity) => Some(ClientIdEntity(Sanitizer.desanitize(client.name))) - case _ => sanitizedClientId + case _ => sanitizedClientEntity } val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 483f8b9fa7409..f5006611485c7 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -149,13 +149,13 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag } // Convert entity into Options with sanitized values for QuotaManagers - val (userEntity, sanitizedClientId) = transferToClientQuotaEntity(quotaEntity) + val (userEntity, sanitizedClientEntity) = transferToClientQuotaEntity(quotaEntity) val quotaValue = newValue.map(new Quota(_, true)) try { manager.updateQuota( userEntity = userEntity, - sanitizedClientId = sanitizedClientId, + sanitizedClientEntity = sanitizedClientEntity, quota = quotaValue ) } catch { @@ -167,7 +167,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag object ClientQuotaMetadataManager { def transferToClientQuotaEntity(quotaEntity: QuotaEntity): (Option[BaseUserEntity], Option[ClientQuotaConfigEntity]) = { - val (userEntity, sanitizedClientId) = quotaEntity match { + val (userEntity, sanitizedClientEntity) = quotaEntity match { case UserEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) case DefaultUserEntity => @@ -186,6 +186,6 @@ object ClientQuotaMetadataManager { (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } - (userEntity, sanitizedClientId) + (userEntity, sanitizedClientEntity) } } \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index e42e68ae16ad4..2b34f506fd0a5 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -21,7 +21,6 @@ import kafka.server.ClientQuotaManager.BaseUserEntity import java.net.InetAddress import org.apache.kafka.common.metrics.Quota import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.common.utils.Sanitizer import org.apache.kafka.server.config.ClientQuotaManagerConfig import org.apache.kafka.network.Session import org.apache.kafka.server.quota.QuotaType @@ -38,12 +37,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientEntity, Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( client2.configUser, - client2.sanitizedConfigClientId, + client2.sanitizedConfigClientEntity, Some(new Quota(4000, true)) ) @@ -62,7 +61,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientEntity, Some(new Quota(3000, true)) ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") @@ -73,7 +72,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientEntity, Some(new Quota(500, true)) ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") @@ -84,12 +83,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientId, + client1.sanitizedConfigClientEntity, None ) clientQuotaManager.updateQuota( defaultConfigClient.configUser, - defaultConfigClient.sanitizedConfigClientId, + defaultConfigClient.sanitizedConfigClientEntity, Some(new Quota(4000, true)) ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") @@ -102,20 +101,6 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } } - /** - * Tests parsing for quotas. - * Quota overrides persisted in ZooKeeper in /config/users/, default persisted in /config/users/ - */ - @Test - def testUserQuotaParsing(): Unit = { - val client1 = UserClient("User1", "p1", Some(ClientQuotaManager.UserEntity("User1")), None) - val client2 = UserClient("User2", "p2", Some(ClientQuotaManager.UserEntity("User2")), None) - val randomClient = UserClient("RandomUser", "random-client-id", None, None) - val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultUserEntity), None) - val config = new ClientQuotaManagerConfig() - testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) - } - /** * Tests parsing for quotas when client-id default quota properties are set. */ @@ -516,11 +501,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } } - private case class UserClient(user: String, clientId: String, configUser: Option[BaseUserEntity] = None, configClientId: Option[String] = None) { - // The class under test expects only sanitized client configs. We pass both the default value (which should not be - // sanitized to ensure it remains unique) and non-default values, so we need to take care in generating the sanitized - // client ID - def sanitizedConfigClientId = configClientId.map(x => Sanitizer.sanitize(x)) - .map(ClientQuotaManager.ClientIdEntity) - } + private case class UserClient( + user: String, clientId: String, + configUser: Option[BaseUserEntity] = None, + sanitizedConfigClientEntity: Option[ClientQuotaManager.ClientIdEntity] = None + ) } From b52aaedd81984625ad500902ca55c11246573420 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 3 Feb 2025 19:44:19 +0800 Subject: [PATCH 16/20] addressed by comment --- core/src/main/scala/kafka/admin/ConfigCommand.scala | 12 +++++++++++- .../main/scala/kafka/server/ClientQuotaManager.scala | 8 ++++---- .../scala/kafka/server/DynamicBrokerConfig.scala | 7 +++++++ core/src/main/scala/kafka/server/DynamicConfig.scala | 9 --------- .../server/metadata/ClientQuotaMetadataManager.scala | 3 +-- 5 files changed, 23 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 1cf2a800de989..edf0ff3bb6fa7 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -36,6 +36,7 @@ import org.apache.kafka.server.config.{ConfigType, QuotaConfig} import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils} import org.apache.kafka.storage.internals.log.LogConfig +import java.net.{InetAddress, UnknownHostException} import scala.jdk.CollectionConverters._ import scala.collection._ @@ -649,7 +650,7 @@ object ConfigCommand extends Logging { if (hasEntityName && entityTypeVals.contains(ConfigType.IP)) { Seq(entityName, ip).filter(options.has(_)).map(options.valueOf(_)).foreach { ipEntity => - if (!DynamicConfig.Ip.isValidIpEntity(ipEntity)) + if (!isValidIpEntity(ipEntity)) throw new IllegalArgumentException(s"The entity name for ${entityTypeVals.head} must be a valid IP or resolvable host, but it is: $ipEntity") } } @@ -688,4 +689,13 @@ object ConfigCommand extends Logging { } } } + + def isValidIpEntity(ip: String): Boolean = { + try { + InetAddress.getByName(ip) + } catch { + case _: UnknownHostException => return false + } + true + } } diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 224bd9b8b7e09..f648bd334f81f 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -55,7 +55,7 @@ object QuotaTypes { object ClientQuotaManager { // Purge sensors after 1 hour of inactivity val InactiveSensorExpirationTimeSeconds = 3600 - private val DefaultString = "" + private val DefaultName = "" val DefaultClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity)) val DefaultUserQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None) val DefaultUserClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity)) @@ -76,13 +76,13 @@ object ClientQuotaManager { case object DefaultUserEntity extends BaseUserEntity { override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_USER - override def name: String = DefaultString + override def name: String = DefaultName override def toString: String = "default user" } case object DefaultClientIdEntity extends ClientQuotaEntity.ConfigEntity { override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_CLIENT_ID - override def name: String = DefaultString + override def name: String = DefaultName override def toString: String = "default client-id" } @@ -93,7 +93,7 @@ object ClientQuotaManager { def sanitizedUser: String = userEntity.map { case entity: UserEntity => entity.sanitizedUser - case DefaultUserEntity => DefaultString + case DefaultUserEntity => DefaultName }.getOrElse("") def clientId: String = clientIdEntity.map(_.name).getOrElse("") diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 06e7be6111c64..aa139bf9886be 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -48,6 +48,13 @@ import scala.collection._ import scala.jdk.CollectionConverters._ /** + * Dynamic broker configurations may be defined at two levels: + *
      + *
    • Per-broker configs persisted at /configs/brokers/{brokerId}: These can be described/altered + * using AdminClient using the resource name brokerId.
    • + *
    • Cluster-wide defaults persisted at /configs/brokers/<default>: These can be described/altered + * using AdminClient using an empty resource name.
    • + *
    * The order of precedence for broker configs is: *
      *
    1. STATIC_BROKER_CONFIG: properties that broker is started up with, typically from server.properties file
    2. diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index f6452de676fad..aadee46baf5eb 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -83,15 +83,6 @@ object DynamicConfig { def names: util.Set[String] = ipConfigs.names def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false) - - def isValidIpEntity(ip: String): Boolean = { - try { - InetAddress.getByName(ip) - } catch { - case _: UnknownHostException => return false - } - true - } } object ClientMetrics { diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index f5006611485c7..d7410200585b3 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -167,7 +167,7 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag object ClientQuotaMetadataManager { def transferToClientQuotaEntity(quotaEntity: QuotaEntity): (Option[BaseUserEntity], Option[ClientQuotaConfigEntity]) = { - val (userEntity, sanitizedClientEntity) = quotaEntity match { + quotaEntity match { case UserEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None) case DefaultUserEntity => @@ -186,6 +186,5 @@ object ClientQuotaMetadataManager { (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } - (userEntity, sanitizedClientEntity) } } \ No newline at end of file From a201ae955212ef522313be6a0be7c6b3125d134f Mon Sep 17 00:00:00 2001 From: m1a2st Date: Mon, 3 Feb 2025 19:47:03 +0800 Subject: [PATCH 17/20] remove unused import --- core/src/main/scala/kafka/server/DynamicConfig.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index aadee46baf5eb..7a401ec1eb426 100644 --- a/core/src/main/scala/kafka/server/DynamicConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicConfig.scala @@ -19,7 +19,6 @@ package kafka.server import kafka.server.DynamicBrokerConfig.AllDynamicConfigs -import java.net.{InetAddress, UnknownHostException} import java.util.Properties import org.apache.kafka.common.config.ConfigDef import org.apache.kafka.coordinator.group.GroupConfig From 13d9b7f866bc20d40f462d2a1b254ae6fea85287 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 4 Feb 2025 20:26:49 +0800 Subject: [PATCH 18/20] update the document --- .../src/main/scala/kafka/server/DynamicBrokerConfig.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala index 69889a440d424..aed5c57cbeb1d 100755 --- a/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala +++ b/core/src/main/scala/kafka/server/DynamicBrokerConfig.scala @@ -48,10 +48,10 @@ import scala.jdk.CollectionConverters._ /** * Dynamic broker configurations may be defined at two levels: *
        - *
      • Per-broker configs persisted at /configs/brokers/{brokerId}: These can be described/altered - * using AdminClient using the resource name brokerId.
      • - *
      • Cluster-wide defaults persisted at /configs/brokers/<default>: These can be described/altered - * using AdminClient using an empty resource name.
      • + *
      • Per-broker configurations are persisted at the controller and can be described + * or altered using AdminClient with the resource name brokerId.
      • + *
      • Cluster-wide default configurations are persisted at the cluster level and can be + * described or altered using AdminClient with an empty resource name.
      • *
      * The order of precedence for broker configs is: *
        From ed6353b05b5ee56598ffbe676d5f0c0951a4685e Mon Sep 17 00:00:00 2001 From: m1a2st Date: Wed, 5 Feb 2025 22:14:54 +0800 Subject: [PATCH 19/20] addressed by comments --- .../scala/kafka/server/ClientQuotaManager.scala | 14 +++++++------- .../metadata/ClientQuotaMetadataManager.scala | 10 +++++----- .../kafka/server/ClientQuotaManagerTest.scala | 17 +++++++++-------- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index f648bd334f81f..ed294d4aac33d 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -403,13 +403,13 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, * Overrides quotas for , or or the dynamic defaults * for any of these levels. * - * @param userEntity user to override if quota applies to or - * @param sanitizedClientEntity sanitized client entity to override if quota applies to or - * @param quota custom quota to apply or None if quota override is being removed + * @param userEntity user to override if quota applies to or + * @param clientEntity sanitized client entity to override if quota applies to or + * @param quota custom quota to apply or None if quota override is being removed */ def updateQuota( userEntity: Option[BaseUserEntity], - sanitizedClientEntity: Option[ClientQuotaEntity.ConfigEntity], + clientEntity: Option[ClientQuotaEntity.ConfigEntity], quota: Option[Quota] ): Unit = { /* @@ -421,9 +421,9 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val clientIdEntity = sanitizedClientEntity match { - case Some(client: ClientIdEntity) => Some(ClientIdEntity(Sanitizer.desanitize(client.name))) - case _ => sanitizedClientEntity + val clientIdEntity = clientEntity match { + case Some(client: ClientIdEntity) => Some(ClientIdEntity(client.name)) + case _ => clientEntity } val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) diff --git a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index d7410200585b3..8fae9941b4112 100644 --- a/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala +++ b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala @@ -149,13 +149,13 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag } // Convert entity into Options with sanitized values for QuotaManagers - val (userEntity, sanitizedClientEntity) = transferToClientQuotaEntity(quotaEntity) + val (userEntity, clientEntity) = transferToClientQuotaEntity(quotaEntity) val quotaValue = newValue.map(new Quota(_, true)) try { manager.updateQuota( userEntity = userEntity, - sanitizedClientEntity = sanitizedClientEntity, + clientEntity = clientEntity, quota = quotaValue ) } catch { @@ -173,15 +173,15 @@ object ClientQuotaMetadataManager { case DefaultUserEntity => (Some(ClientQuotaManager.DefaultUserEntity), None) case ClientIdEntity(clientId) => - (None, Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + (None, Some(ClientQuotaManager.ClientIdEntity(clientId))) case DefaultClientIdEntity => (None, Some(ClientQuotaManager.DefaultClientIdEntity)) case ExplicitUserExplicitClientIdEntity(user, clientId) => - (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(clientId))) case ExplicitUserDefaultClientIdEntity(user) => (Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultClientIdEntity)) case DefaultUserExplicitClientIdEntity(clientId) => - (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(Sanitizer.sanitize(clientId)))) + (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(clientId))) case DefaultUserDefaultClientIdEntity => (Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 2b34f506fd0a5..6c268d3c3fbd9 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -37,12 +37,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 1: Update the quota. Assert that the new quota value is returned clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientEntity, + client1.configClientEntity, Some(new Quota(2000, true)) ) clientQuotaManager.updateQuota( client2.configUser, - client2.sanitizedConfigClientEntity, + client2.configClientEntity, Some(new Quota(4000, true)) ) @@ -61,7 +61,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // p1 should not longer be throttled after the quota change clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientEntity, + client1.configClientEntity, Some(new Quota(3000, true)) ) assertEquals(3000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (3000)") @@ -72,7 +72,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 3: Change quota back to default. Should be throttled again clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientEntity, + client1.configClientEntity, Some(new Quota(500, true)) ) assertEquals(500, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the default value (500)") @@ -83,12 +83,12 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { // Case 4: Set high default quota, remove p1 quota. p1 should no longer be throttled clientQuotaManager.updateQuota( client1.configUser, - client1.sanitizedConfigClientEntity, + client1.configClientEntity, None ) clientQuotaManager.updateQuota( defaultConfigClient.configUser, - defaultConfigClient.sanitizedConfigClientEntity, + defaultConfigClient.configClientEntity, Some(new Quota(4000, true)) ) assertEquals(4000, clientQuotaManager.quota(client1.user, client1.clientId).bound, 0.0, "Should return the newly overridden value (4000)") @@ -502,8 +502,9 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { } private case class UserClient( - user: String, clientId: String, + user: String, + clientId: String, configUser: Option[BaseUserEntity] = None, - sanitizedConfigClientEntity: Option[ClientQuotaManager.ClientIdEntity] = None + configClientEntity: Option[ClientQuotaManager.ClientIdEntity] = None ) } From d098405dde55db19e5105498bc16601d15ca993d Mon Sep 17 00:00:00 2001 From: m1a2st Date: Thu, 6 Feb 2025 14:17:45 +0800 Subject: [PATCH 20/20] update the test --- .../main/scala/kafka/server/ClientQuotaManager.scala | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index ed294d4aac33d..d8346e6ab85ea 100644 --- a/core/src/main/scala/kafka/server/ClientQuotaManager.scala +++ b/core/src/main/scala/kafka/server/ClientQuotaManager.scala @@ -421,26 +421,21 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, */ lock.writeLock().lock() try { - val clientIdEntity = clientEntity match { - case Some(client: ClientIdEntity) => Some(ClientIdEntity(client.name)) - case _ => clientEntity - } - - val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) + val quotaEntity = KafkaQuotaEntity(userEntity, clientEntity) if (userEntity.nonEmpty) { if (quotaEntity.clientIdEntity.nonEmpty) quotaTypesEnabled |= QuotaTypes.UserClientIdQuotaEnabled else quotaTypesEnabled |= QuotaTypes.UserQuotaEnabled - } else if (clientIdEntity.nonEmpty) + } else if (clientEntity.nonEmpty) quotaTypesEnabled |= QuotaTypes.ClientIdQuotaEnabled quota match { case Some(newQuota) => quotaCallback.updateQuota(clientQuotaType, quotaEntity, newQuota.bound) case None => quotaCallback.removeQuota(clientQuotaType, quotaEntity) } - val updatedEntity = if (userEntity.contains(DefaultUserEntity) || clientIdEntity.contains(DefaultClientIdEntity)) + val updatedEntity = if (userEntity.contains(DefaultUserEntity) || clientEntity.contains(DefaultClientIdEntity)) None // more than one entity may need updating, so `updateQuotaMetricConfigs` will go through all metrics else Some(quotaEntity)