Skip to content

Commit 59351d4

Browse files
m1a2stmanoj-mathivanan
authored andcommitted
KAFKA-18530 Remove ZooKeeperInternals (apache#18641)
Since zk has been removed in 4.0, config handlers no longer need to handle the "<default>" value. This PR streamlines the config update process by eliminating the unnecessary string checks for "<default>" Reviewers: Christo Lolov <[email protected]>, Ismael Juma <[email protected]>, Chia-Ping Tsai <[email protected]>
1 parent 4cd8587 commit 59351d4

14 files changed

+332
-222
lines changed

core/src/main/scala/kafka/admin/ConfigCommand.scala

+11-1
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,7 @@ import org.apache.kafka.server.config.{ConfigType, QuotaConfig}
3636
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils}
3737
import org.apache.kafka.storage.internals.log.LogConfig
3838

39+
import java.net.{InetAddress, UnknownHostException}
3940
import scala.jdk.CollectionConverters._
4041
import scala.collection._
4142

@@ -649,7 +650,7 @@ object ConfigCommand extends Logging {
649650

650651
if (hasEntityName && entityTypeVals.contains(ConfigType.IP)) {
651652
Seq(entityName, ip).filter(options.has(_)).map(options.valueOf(_)).foreach { ipEntity =>
652-
if (!DynamicConfig.Ip.isValidIpEntity(ipEntity))
653+
if (!isValidIpEntity(ipEntity))
653654
throw new IllegalArgumentException(s"The entity name for ${entityTypeVals.head} must be a valid IP or resolvable host, but it is: $ipEntity")
654655
}
655656
}
@@ -688,4 +689,13 @@ object ConfigCommand extends Logging {
688689
}
689690
}
690691
}
692+
693+
def isValidIpEntity(ip: String): Boolean = {
694+
try {
695+
InetAddress.getByName(ip)
696+
} catch {
697+
case _: UnknownHostException => return false
698+
}
699+
true
700+
}
691701
}

core/src/main/scala/kafka/server/ClientQuotaManager.scala

+19-25
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ import org.apache.kafka.common.metrics.Metrics
2929
import org.apache.kafka.common.metrics.stats.{Avg, CumulativeSum, Rate}
3030
import org.apache.kafka.common.security.auth.KafkaPrincipal
3131
import org.apache.kafka.common.utils.{Sanitizer, Time}
32-
import org.apache.kafka.server.config.{ClientQuotaManagerConfig, ZooKeeperInternals}
32+
import org.apache.kafka.server.config.ClientQuotaManagerConfig
3333
import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaEntity, ClientQuotaType, QuotaType, QuotaUtils, SensorAccess, ThrottleCallback, ThrottledChannel}
3434
import org.apache.kafka.server.util.ShutdownableThread
3535
import org.apache.kafka.network.Session
@@ -55,7 +55,7 @@ object QuotaTypes {
5555
object ClientQuotaManager {
5656
// Purge sensors after 1 hour of inactivity
5757
val InactiveSensorExpirationTimeSeconds = 3600
58-
58+
private val DefaultName = "<default>"
5959
val DefaultClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(None, Some(DefaultClientIdEntity))
6060
val DefaultUserQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), None)
6161
val DefaultUserClientIdQuotaEntity: KafkaQuotaEntity = KafkaQuotaEntity(Some(DefaultUserEntity), Some(DefaultClientIdEntity))
@@ -76,13 +76,13 @@ object ClientQuotaManager {
7676

7777
case object DefaultUserEntity extends BaseUserEntity {
7878
override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_USER
79-
override def name: String = ZooKeeperInternals.DEFAULT_STRING
79+
override def name: String = DefaultName
8080
override def toString: String = "default user"
8181
}
8282

8383
case object DefaultClientIdEntity extends ClientQuotaEntity.ConfigEntity {
8484
override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_CLIENT_ID
85-
override def name: String = ZooKeeperInternals.DEFAULT_STRING
85+
override def name: String = DefaultName
8686
override def toString: String = "default client-id"
8787
}
8888

@@ -93,7 +93,7 @@ object ClientQuotaManager {
9393

9494
def sanitizedUser: String = userEntity.map {
9595
case entity: UserEntity => entity.sanitizedUser
96-
case DefaultUserEntity => ZooKeeperInternals.DEFAULT_STRING
96+
case DefaultUserEntity => DefaultName
9797
}.getOrElse("")
9898

9999
def clientId: String = clientIdEntity.map(_.name).getOrElse("")
@@ -260,7 +260,7 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
260260
*/
261261
def unrecordQuotaSensor(request: RequestChannel.Request, value: Double, timeMs: Long): Unit = {
262262
val clientSensors = getOrCreateQuotaSensors(request.session, request.header.clientId)
263-
clientSensors.quotaSensor.record(value * (-1), timeMs, false)
263+
clientSensors.quotaSensor.record(value * -1, timeMs, false)
264264
}
265265

266266
/**
@@ -403,12 +403,15 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
403403
* Overrides quotas for <user>, <client-id> or <user, client-id> or the dynamic defaults
404404
* for any of these levels.
405405
*
406-
* @param sanitizedUser user to override if quota applies to <user> or <user, client-id>
407-
* @param clientId client to override if quota applies to <client-id> or <user, client-id>
408-
* @param sanitizedClientId sanitized client ID to override if quota applies to <client-id> or <user, client-id>
409-
* @param quota custom quota to apply or None if quota override is being removed
406+
* @param userEntity user to override if quota applies to <user> or <user, client-id>
407+
* @param clientEntity sanitized client entity to override if quota applies to <client-id> or <user, client-id>
408+
* @param quota custom quota to apply or None if quota override is being removed
410409
*/
411-
def updateQuota(sanitizedUser: Option[String], clientId: Option[String], sanitizedClientId: Option[String], quota: Option[Quota]): Unit = {
410+
def updateQuota(
411+
userEntity: Option[BaseUserEntity],
412+
clientEntity: Option[ClientQuotaEntity.ConfigEntity],
413+
quota: Option[Quota]
414+
): Unit = {
412415
/*
413416
* Acquire the write lock to apply changes in the quota objects.
414417
* This method changes the quota in the overriddenQuota map and applies the update on the actual KafkaMetric object (if it exists).
@@ -418,29 +421,21 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
418421
*/
419422
lock.writeLock().lock()
420423
try {
421-
val userEntity = sanitizedUser.map {
422-
case ZooKeeperInternals.DEFAULT_STRING => DefaultUserEntity
423-
case user => UserEntity(user)
424-
}
425-
val clientIdEntity = sanitizedClientId.map {
426-
case ZooKeeperInternals.DEFAULT_STRING => DefaultClientIdEntity
427-
case _ => ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))
428-
}
429-
val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity)
424+
val quotaEntity = KafkaQuotaEntity(userEntity, clientEntity)
430425

431426
if (userEntity.nonEmpty) {
432427
if (quotaEntity.clientIdEntity.nonEmpty)
433428
quotaTypesEnabled |= QuotaTypes.UserClientIdQuotaEnabled
434429
else
435430
quotaTypesEnabled |= QuotaTypes.UserQuotaEnabled
436-
} else if (clientIdEntity.nonEmpty)
431+
} else if (clientEntity.nonEmpty)
437432
quotaTypesEnabled |= QuotaTypes.ClientIdQuotaEnabled
438433

439434
quota match {
440435
case Some(newQuota) => quotaCallback.updateQuota(clientQuotaType, quotaEntity, newQuota.bound)
441436
case None => quotaCallback.removeQuota(clientQuotaType, quotaEntity)
442437
}
443-
val updatedEntity = if (userEntity.contains(DefaultUserEntity) || clientIdEntity.contains(DefaultClientIdEntity))
438+
val updatedEntity = if (userEntity.contains(DefaultUserEntity) || clientEntity.contains(DefaultClientIdEntity))
444439
None // more than one entity may need updating, so `updateQuotaMetricConfigs` will go through all metrics
445440
else
446441
Some(quotaEntity)
@@ -452,9 +447,8 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
452447
}
453448

454449
/**
455-
* Updates metrics configs. This is invoked when quota configs are updated in ZooKeeper
456-
* or when partitions leaders change and custom callbacks that implement partition-based quotas
457-
* have updated quotas.
450+
* Updates metrics configs. This is invoked when quota configs are updated when partitions leaders change
451+
* and custom callbacks that implement partition-based quotas have updated quotas.
458452
*
459453
* @param updatedQuotaEntity If set to one entity and quotas have only been enabled at one
460454
* level, then an optimized update is performed with a single metric update. If None is provided,

core/src/main/scala/kafka/server/ConfigHandler.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.util.{Collections, Properties}
2121
import kafka.log.UnifiedLog
2222
import kafka.server.QuotaFactory.QuotaManagers
2323
import kafka.utils.Logging
24-
import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals}
24+
import org.apache.kafka.server.config.QuotaConfig
2525
import org.apache.kafka.common.metrics.Quota._
2626
import org.apache.kafka.coordinator.group.GroupCoordinator
2727
import org.apache.kafka.server.ClientMetricsManager
@@ -146,7 +146,7 @@ class TopicConfigHandler(private val replicaManager: ReplicaManager,
146146
class BrokerConfigHandler(private val brokerConfig: KafkaConfig,
147147
private val quotaManagers: QuotaManagers) extends ConfigHandler with Logging {
148148
def processConfigChanges(brokerId: String, properties: Properties): Unit = {
149-
if (brokerId == ZooKeeperInternals.DEFAULT_STRING)
149+
if (brokerId.isEmpty)
150150
brokerConfig.dynamicConfig.updateDefaultConfig(properties)
151151
else if (brokerConfig.brokerId == brokerId.trim.toInt) {
152152
brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties)

core/src/main/scala/kafka/server/DynamicBrokerConfig.scala

+6-6
Original file line numberDiff line numberDiff line change
@@ -46,12 +46,12 @@ import scala.collection._
4646
import scala.jdk.CollectionConverters._
4747

4848
/**
49-
* Dynamic broker configurations are stored in ZooKeeper and may be defined at two levels:
49+
* Dynamic broker configurations may be defined at two levels:
5050
* <ul>
51-
* <li>Per-broker configs persisted at <tt>/configs/brokers/{brokerId}</tt>: These can be described/altered
52-
* using AdminClient using the resource name brokerId.</li>
53-
* <li>Cluster-wide defaults persisted at <tt>/configs/brokers/&lt;default&gt;</tt>: These can be described/altered
54-
* using AdminClient using an empty resource name.</li>
51+
* <li>Per-broker configurations are persisted at the controller and can be described
52+
* or altered using AdminClient with the resource name brokerId.</li>
53+
* <li>Cluster-wide default configurations are persisted at the cluster level and can be
54+
* described or altered using AdminClient with an empty resource name.</li>
5555
* </ul>
5656
* The order of precedence for broker configs is:
5757
* <ol>
@@ -367,7 +367,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
367367
error(s"$errorMessage: $invalidPropNames")
368368
}
369369
}
370-
removeInvalidProps(nonDynamicConfigs(props), "Non-dynamic configs configured in ZooKeeper will be ignored")
370+
removeInvalidProps(nonDynamicConfigs(props), "Non-dynamic configs will be ignored")
371371
removeInvalidProps(securityConfigsWithoutListenerPrefix(props),
372372
"Security configs can be dynamically updated only using listener prefix, base configs will be ignored")
373373
if (!perBrokerConfig)

core/src/main/scala/kafka/server/DynamicConfig.scala

+1-13
Original file line numberDiff line numberDiff line change
@@ -19,11 +19,10 @@ package kafka.server
1919

2020
import kafka.server.DynamicBrokerConfig.AllDynamicConfigs
2121

22-
import java.net.{InetAddress, UnknownHostException}
2322
import java.util.Properties
2423
import org.apache.kafka.common.config.ConfigDef
2524
import org.apache.kafka.coordinator.group.GroupConfig
26-
import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals}
25+
import org.apache.kafka.server.config.QuotaConfig
2726

2827
import java.util
2928
import scala.jdk.CollectionConverters._
@@ -83,17 +82,6 @@ object DynamicConfig {
8382
def names: util.Set[String] = ipConfigs.names
8483

8584
def validate(props: Properties): util.Map[String, AnyRef] = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false)
86-
87-
def isValidIpEntity(ip: String): Boolean = {
88-
if (ip != ZooKeeperInternals.DEFAULT_STRING) {
89-
try {
90-
InetAddress.getByName(ip)
91-
} catch {
92-
case _: UnknownHostException => return false
93-
}
94-
}
95-
true
96-
}
9785
}
9886

9987
object ClientMetrics {

core/src/main/scala/kafka/server/metadata/ClientQuotaMetadataManager.scala

+35-16
Original file line numberDiff line numberDiff line change
@@ -18,15 +18,19 @@
1818
package kafka.server.metadata
1919

2020
import kafka.network.ConnectionQuotas
21+
import kafka.server.ClientQuotaManager
22+
import kafka.server.ClientQuotaManager.BaseUserEntity
2123
import kafka.server.QuotaFactory.QuotaManagers
24+
import kafka.server.metadata.ClientQuotaMetadataManager.transferToClientQuotaEntity
2225
import kafka.utils.Logging
2326
import org.apache.kafka.common.metrics.Quota
2427
import org.apache.kafka.common.quota.ClientQuotaEntity
2528
import org.apache.kafka.common.utils.Sanitizer
29+
import org.apache.kafka.server.quota.ClientQuotaEntity.{ConfigEntity => ClientQuotaConfigEntity}
2630

2731
import java.net.{InetAddress, UnknownHostException}
2832
import org.apache.kafka.image.{ClientQuotaDelta, ClientQuotasDelta}
29-
import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals}
33+
import org.apache.kafka.server.config.QuotaConfig
3034

3135
import scala.jdk.OptionConverters.RichOptionalDouble
3236

@@ -145,27 +149,42 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
145149
}
146150

147151
// Convert entity into Options with sanitized values for QuotaManagers
148-
val (sanitizedUser, sanitizedClientId) = quotaEntity match {
149-
case UserEntity(user) => (Some(Sanitizer.sanitize(user)), None)
150-
case DefaultUserEntity => (Some(ZooKeeperInternals.DEFAULT_STRING), None)
151-
case ClientIdEntity(clientId) => (None, Some(Sanitizer.sanitize(clientId)))
152-
case DefaultClientIdEntity => (None, Some(ZooKeeperInternals.DEFAULT_STRING))
153-
case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(Sanitizer.sanitize(user)), Some(Sanitizer.sanitize(clientId)))
154-
case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(ZooKeeperInternals.DEFAULT_STRING))
155-
case DefaultUserExplicitClientIdEntity(clientId) => (Some(ZooKeeperInternals.DEFAULT_STRING), Some(Sanitizer.sanitize(clientId)))
156-
case DefaultUserDefaultClientIdEntity => (Some(ZooKeeperInternals.DEFAULT_STRING), Some(ZooKeeperInternals.DEFAULT_STRING))
157-
case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here")
158-
}
152+
val (userEntity, clientEntity) = transferToClientQuotaEntity(quotaEntity)
159153

160154
val quotaValue = newValue.map(new Quota(_, true))
161155
try {
162156
manager.updateQuota(
163-
sanitizedUser = sanitizedUser,
164-
clientId = sanitizedClientId.map(Sanitizer.desanitize),
165-
sanitizedClientId = sanitizedClientId,
166-
quota = quotaValue)
157+
userEntity = userEntity,
158+
clientEntity = clientEntity,
159+
quota = quotaValue
160+
)
167161
} catch {
168162
case t: Throwable => error(s"Failed to update user-client quota $quotaEntity", t)
169163
}
170164
}
171165
}
166+
167+
object ClientQuotaMetadataManager {
168+
169+
def transferToClientQuotaEntity(quotaEntity: QuotaEntity): (Option[BaseUserEntity], Option[ClientQuotaConfigEntity]) = {
170+
quotaEntity match {
171+
case UserEntity(user) =>
172+
(Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), None)
173+
case DefaultUserEntity =>
174+
(Some(ClientQuotaManager.DefaultUserEntity), None)
175+
case ClientIdEntity(clientId) =>
176+
(None, Some(ClientQuotaManager.ClientIdEntity(clientId)))
177+
case DefaultClientIdEntity =>
178+
(None, Some(ClientQuotaManager.DefaultClientIdEntity))
179+
case ExplicitUserExplicitClientIdEntity(user, clientId) =>
180+
(Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.ClientIdEntity(clientId)))
181+
case ExplicitUserDefaultClientIdEntity(user) =>
182+
(Some(ClientQuotaManager.UserEntity(Sanitizer.sanitize(user))), Some(ClientQuotaManager.DefaultClientIdEntity))
183+
case DefaultUserExplicitClientIdEntity(clientId) =>
184+
(Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity(clientId)))
185+
case DefaultUserDefaultClientIdEntity =>
186+
(Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity))
187+
case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here")
188+
}
189+
}
190+
}

core/src/main/scala/kafka/server/metadata/DynamicConfigPublisher.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ import kafka.utils.Logging
2424
import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, CLIENT_METRICS, GROUP, TOPIC}
2525
import org.apache.kafka.image.loader.LoaderManifest
2626
import org.apache.kafka.image.{MetadataDelta, MetadataImage}
27-
import org.apache.kafka.server.config.{ConfigType, ZooKeeperInternals}
27+
import org.apache.kafka.server.config.ConfigType
2828
import org.apache.kafka.server.fault.FaultHandler
2929

3030

@@ -78,7 +78,7 @@ class DynamicConfigPublisher(
7878
// These are stored in KRaft with an empty name field.
7979
info("Updating cluster configuration : " +
8080
toLoggableProps(resource, props).mkString(","))
81-
nodeConfigHandler.processConfigChanges(ZooKeeperInternals.DEFAULT_STRING, props)
81+
nodeConfigHandler.processConfigChanges(resource.name(), props)
8282
} catch {
8383
case t: Throwable => faultHandler.handleFault("Error updating " +
8484
s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +

core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala

+5-2
Original file line numberDiff line numberDiff line change
@@ -934,8 +934,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
934934
// non-default value to trigger a new metric
935935
val clientId = "test-client-1"
936936
servers.foreach { server =>
937-
server.quotaManagers.produce.updateQuota(None, Some(clientId), Some(clientId),
938-
Some(Quota.upperBound(10000000)))
937+
server.quotaManagers.produce.updateQuota(
938+
None,
939+
Some(ClientQuotaManager.ClientIdEntity(clientId)),
940+
Some(Quota.upperBound(10000000))
941+
)
939942
}
940943
val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol, clientId)
941944
TestUtils.waitUntilTrue(() => consumerThread.received >= 5, "Messages not sent")

core/src/test/scala/kafka/server/metadata/ClientQuotaMetadataManagerTest.scala

+41-1
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,9 @@
1616
*/
1717
package kafka.server.metadata
1818

19+
import kafka.server.ClientQuotaManager
1920
import org.apache.kafka.image.ClientQuotaDelta
20-
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertThrows}
21+
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertThrows}
2122
import org.junit.jupiter.api.Test
2223
import org.junit.jupiter.api.function.Executable
2324

@@ -33,4 +34,43 @@ class ClientQuotaMetadataManagerTest {
3334
assertDoesNotThrow { new Executable { def execute(): Unit = manager.handleIpQuota(IpEntity("192.168.1.1"), new ClientQuotaDelta(null)) } }
3435
assertDoesNotThrow { new Executable { def execute(): Unit = manager.handleIpQuota(IpEntity("2001:db8::1"), new ClientQuotaDelta(null)) } }
3536
}
37+
38+
@Test
39+
def testTransferToClientQuotaEntity(): Unit = {
40+
41+
assertThrows(classOf[IllegalStateException],() => ClientQuotaMetadataManager.transferToClientQuotaEntity(IpEntity("a")))
42+
assertThrows(classOf[IllegalStateException],() => ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultIpEntity))
43+
assertEquals(
44+
(Some(ClientQuotaManager.UserEntity("user")), None),
45+
ClientQuotaMetadataManager.transferToClientQuotaEntity(UserEntity("user"))
46+
)
47+
assertEquals(
48+
(Some(ClientQuotaManager.DefaultUserEntity), None),
49+
ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserEntity)
50+
)
51+
assertEquals(
52+
(None, Some(ClientQuotaManager.ClientIdEntity("client"))),
53+
ClientQuotaMetadataManager.transferToClientQuotaEntity(ClientIdEntity("client"))
54+
)
55+
assertEquals(
56+
(None, Some(ClientQuotaManager.DefaultClientIdEntity)),
57+
ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultClientIdEntity)
58+
)
59+
assertEquals(
60+
(Some(ClientQuotaManager.UserEntity("user")), Some(ClientQuotaManager.ClientIdEntity("client"))),
61+
ClientQuotaMetadataManager.transferToClientQuotaEntity(ExplicitUserExplicitClientIdEntity("user", "client"))
62+
)
63+
assertEquals(
64+
(Some(ClientQuotaManager.UserEntity("user")), Some(ClientQuotaManager.DefaultClientIdEntity)),
65+
ClientQuotaMetadataManager.transferToClientQuotaEntity(ExplicitUserDefaultClientIdEntity("user"))
66+
)
67+
assertEquals(
68+
(Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.ClientIdEntity("client"))),
69+
ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserExplicitClientIdEntity("client"))
70+
)
71+
assertEquals(
72+
(Some(ClientQuotaManager.DefaultUserEntity), Some(ClientQuotaManager.DefaultClientIdEntity)),
73+
ClientQuotaMetadataManager.transferToClientQuotaEntity(DefaultUserDefaultClientIdEntity)
74+
)
75+
}
3676
}

0 commit comments

Comments
 (0)