From 9391127991866324d5247f61b24fd4ce30435733 Mon Sep 17 00:00:00 2001 From: m1a2st Date: Tue, 21 Jan 2025 21:39:49 +0800 Subject: [PATCH] update the DynamicBrokerConfig comment --- .../kafka/server/ClientQuotaManager.scala | 12 +++--- .../scala/kafka/server/ConfigHandler.scala | 4 +- .../scala/kafka/server/DynamicConfig.scala | 12 +++--- .../metadata/ClientQuotaMetadataManager.scala | 13 +++--- .../metadata/DynamicConfigPublisher.scala | 14 +------ .../kafka/server/ClientQuotaManagerTest.scala | 40 +++++++++---------- .../server/ClientQuotasRequestTest.scala | 2 +- .../kafka/server/config/KraftInternals.java | 26 ------------ 8 files changed, 41 insertions(+), 82 deletions(-) delete mode 100644 server-common/src/main/java/org/apache/kafka/server/config/KraftInternals.java diff --git a/core/src/main/scala/kafka/server/ClientQuotaManager.scala b/core/src/main/scala/kafka/server/ClientQuotaManager.scala index 787e5b7481f21..9fca5ce56602e 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 = "" 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 = KraftInternals.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 = KraftInternals.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 => KraftInternals.DEFAULT_STRING + case DefaultUserEntity => DefaultString }.getOrElse("") def clientId: String = clientIdEntity.map(_.name).getOrElse("") @@ -419,11 +419,11 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig, lock.writeLock().lock() try { val userEntity = sanitizedUser.map { - case KraftInternals.DEFAULT_STRING => DefaultUserEntity + case DefaultString => DefaultUserEntity case user => UserEntity(user) } val clientIdEntity = sanitizedClientId.map { - case KraftInternals.DEFAULT_STRING => DefaultClientIdEntity + case DefaultString => DefaultClientIdEntity case _ => ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided"))) } val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity) diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 01a2a92fe5dab..53250d0e55742 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -148,9 +148,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 == KraftInternals.DEFAULT_STRING) - brokerConfig.dynamicConfig.updateDefaultConfig(properties) - else if (brokerConfig.brokerId == brokerId.trim.toInt) { + if (brokerConfig.brokerId == brokerId.trim.toInt) { brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties) } val updatedDynamicBrokerConfigs = brokerConfig.dynamicConfig.currentDynamicBrokerConfigs diff --git a/core/src/main/scala/kafka/server/DynamicConfig.scala b/core/src/main/scala/kafka/server/DynamicConfig.scala index 029769b36b7d0..56ff86b018074 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, KraftInternals} +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 != KraftInternals.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/ClientQuotaMetadataManager.scala b/core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala index 7f4560031864f..fcf056081320d 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, KraftInternals} +import org.apache.kafka.server.config.{KraftInternals, QuotaConfig} import scala.jdk.OptionConverters.RichOptionalDouble @@ -147,13 +148,13 @@ 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(KraftInternals.DEFAULT_STRING), None) + case DefaultUserEntity => (Some(ClientQuotaManager.DefaultString), None) case ClientIdEntity(clientId) => (None, Some(Sanitizer.sanitize(clientId))) - case DefaultClientIdEntity => (None, Some(KraftInternals.DEFAULT_STRING)) + case DefaultClientIdEntity => (None, Some(ClientQuotaManager.DefaultString)) case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(Sanitizer.sanitize(user)), Some(Sanitizer.sanitize(clientId))) - case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(KraftInternals.DEFAULT_STRING)) - case DefaultUserExplicitClientIdEntity(clientId) => (Some(KraftInternals.DEFAULT_STRING), Some(Sanitizer.sanitize(clientId))) - case DefaultUserDefaultClientIdEntity => (Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING)) + case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(ClientQuotaManager.DefaultString)) + case DefaultUserExplicitClientIdEntity(clientId) => (Some(ClientQuotaManager.DefaultString), Some(Sanitizer.sanitize(clientId))) + case DefaultUserDefaultClientIdEntity => (Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here") } diff --git a/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala index cfb6ce09a20f4..86398ce08f5c9 100644 --- a/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala +++ b/core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala @@ -72,19 +72,7 @@ class DynamicConfigPublisher( ) case BROKER => dynamicConfigHandlers.get(ConfigType.BROKER).foreach(nodeConfigHandler => - if (resource.name().isEmpty) { - try { - // Apply changes to "cluster configs" (also known as default BROKER configs). - // These are stored in KRaft with an empty name field. - info("Updating cluster configuration : " + - toLoggableProps(resource, props).mkString(",")) - nodeConfigHandler.processConfigChanges(KraftInternals.DEFAULT_STRING, props) - } catch { - case t: Throwable => faultHandler.handleFault("Error updating " + - s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " + - s"in $deltaName", t) - } - } else if (resource.name() == conf.nodeId.toString) { + if (resource.name() == conf.nodeId.toString) { try { // Apply changes to this node's dynamic configuration. info(s"Updating node ${conf.nodeId} with new configuration : " + diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 7b6e53cead081..cd6ac65c75de8 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -85,7 +85,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(KraftInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", None, Some(ClientQuotaManager.DefaultString)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -98,7 +98,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(KraftInternals.DEFAULT_STRING), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -112,7 +112,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(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) val config = new ClientQuotaManagerConfig() testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -125,7 +125,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(KraftInternals.DEFAULT_STRING), None) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), None) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -137,7 +137,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(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING)) + val defaultConfigClient = UserClient("", "", Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString)) testQuotaParsing(config, client1, client2, randomClient, defaultConfigClient) } @@ -168,7 +168,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { assertEquals(Double.MaxValue, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) // Set default quota config - clientQuotaManager.updateQuota(Some(KraftInternals.DEFAULT_STRING), None, None, Some(new Quota(10, true))) + clientQuotaManager.updateQuota(Some(ClientQuotaManager.DefaultString), None, None, Some(new Quota(10, true))) assertEquals(10 * numFullQuotaWindows, clientQuotaManager.getMaxValueInQuotaWindow(userSession, "client1"), 0.01) } finally { clientQuotaManager.shutdown() @@ -186,11 +186,11 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) // Set default quota config - clientQuotaManager.updateQuota(Some(KraftInternals.DEFAULT_STRING), None, None, Some(new Quota(10, true))) + clientQuotaManager.updateQuota(Some(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(KraftInternals.DEFAULT_STRING), None, None, None) + clientQuotaManager.updateQuota(Some(ClientQuotaManager.DefaultString), None, None, None) checkQuota(clientQuotaManager, "userA", "client1", Long.MaxValue, 1000, expectThrottle = false) } finally { clientQuotaManager.shutdown() @@ -241,14 +241,14 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(Some(KraftInternals.DEFAULT_STRING), None, None, Some(new Quota(1000, true))) - clientQuotaManager.updateQuota(None, Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(new Quota(2000, true))) - clientQuotaManager.updateQuota(Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(new Quota(3000, true))) + clientQuotaManager.updateQuota(Some(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.DefaultString), Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), 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(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(new Quota(8000, true))) + clientQuotaManager.updateQuota(Some("userB"), Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), 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))) @@ -266,14 +266,14 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { checkQuota(clientQuotaManager, "userE", "client1", 3000, 2500, expectThrottle = false) // Remove default quota config, revert to default - clientQuotaManager.updateQuota(Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), None) + clientQuotaManager.updateQuota(Some(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(KraftInternals.DEFAULT_STRING), None, None, None) + clientQuotaManager.updateQuota(Some(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) @@ -290,7 +290,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { 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))) checkQuota(clientQuotaManager, "userA", "client6", 11000, 8500, expectThrottle = false) - clientQuotaManager.updateQuota(Some("userA"), Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), Some(new Quota(12000, true))) + clientQuotaManager.updateQuota(Some("userA"), Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(12000, true))) clientQuotaManager.updateQuota(Some("userA"), Some("client6"), Some("client6"), None) checkQuota(clientQuotaManager, "userA", "client6", 12000, 4000, expectThrottle = true) // Throttled due to sum of new and earlier values @@ -304,7 +304,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(KraftInternals.DEFAULT_STRING), Some(KraftInternals.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 +352,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireThrottleTimeSensor(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) @@ -374,7 +374,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { def testExpireQuotaSensors(): Unit = { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") try { - clientQuotaManager.updateQuota(None, Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", "client1", 100) @@ -401,7 +401,7 @@ class ClientQuotaManagerTest extends BaseClientQuotaManagerTest { val clientQuotaManager = new ClientQuotaManager(config, metrics, QuotaType.PRODUCE, time, "") val clientId = "client@#$%" try { - clientQuotaManager.updateQuota(None, Some(KraftInternals.DEFAULT_STRING), Some(KraftInternals.DEFAULT_STRING), + clientQuotaManager.updateQuota(None, Some(ClientQuotaManager.DefaultString), Some(ClientQuotaManager.DefaultString), Some(new Quota(500, true))) maybeRecord(clientQuotaManager, "ANONYMOUS", clientId, 100) @@ -421,6 +421,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 == KraftInternals.DEFAULT_STRING) KraftInternals.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 6876c13270f1d..68d29a972ebb6 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -524,7 +524,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 -> KraftInternals.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/server-common/src/main/java/org/apache/kafka/server/config/KraftInternals.java b/server-common/src/main/java/org/apache/kafka/server/config/KraftInternals.java deleted file mode 100644 index 4bc49385a4f01..0000000000000 --- a/server-common/src/main/java/org/apache/kafka/server/config/KraftInternals.java +++ /dev/null @@ -1,26 +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 KraftInternals { - /** - * In order to make the metrics is consistent with the ZK mode, we need to use the same constant - * to indicate the default. - * For example, default user quotas are stored under /config/users/<default> - */ - public static final String DEFAULT_STRING = ""; -}