Skip to content

Commit da2aa68

Browse files
authored
KAFKA-14588: Move ConfigEntityName to server-common (#14868)
Reviewers: Mickael Maison <[email protected]>, Kamal Chandraprakash <[email protected]>
1 parent 8a76c16 commit da2aa68

18 files changed

+100
-79
lines changed

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

+7-7
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit
2222
import java.util.{Collections, Properties}
2323
import joptsimple._
2424
import kafka.server.DynamicConfig.QuotaConfigs
25-
import kafka.server.{ConfigEntityName, Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig}
25+
import kafka.server.{Defaults, DynamicBrokerConfig, DynamicConfig, KafkaConfig}
2626
import kafka.utils.{Exit, Logging, PasswordEncoder}
2727
import kafka.utils.Implicits._
2828
import kafka.zk.{AdminZkClient, KafkaZkClient}
@@ -35,7 +35,7 @@ import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity,
3535
import org.apache.kafka.common.security.JaasUtils
3636
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter, ScramMechanism}
3737
import org.apache.kafka.common.utils.{Sanitizer, Time, Utils}
38-
import org.apache.kafka.server.config.ConfigType
38+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType}
3939
import org.apache.kafka.server.util.{CommandDefaultOptions, CommandLineUtils}
4040
import org.apache.kafka.storage.internals.log.LogConfig
4141
import org.apache.zookeeper.client.ZKClientConfig
@@ -152,7 +152,7 @@ object ConfigCommand extends Logging {
152152
if (!configsToBeAdded.isEmpty || configsToBeDeleted.nonEmpty) {
153153
validateBrokersNotRunning(entityName, adminZkClient, zkClient, errorMessage)
154154

155-
val perBrokerConfig = entityName != ConfigEntityName.Default
155+
val perBrokerConfig = entityName != ConfigEntityName.DEFAULT
156156
preProcessBrokerConfigs(configsToBeAdded, perBrokerConfig)
157157
}
158158
}
@@ -177,7 +177,7 @@ object ConfigCommand extends Logging {
177177
adminZkClient: AdminZkClient,
178178
zkClient: KafkaZkClient,
179179
errorMessage: String): Unit = {
180-
val perBrokerConfig = entityName != ConfigEntityName.Default
180+
val perBrokerConfig = entityName != ConfigEntityName.DEFAULT
181181
val info = "Broker configuration operations using ZooKeeper are only supported if the affected broker(s) are not running."
182182
if (perBrokerConfig) {
183183
adminZkClient.parseBroker(entityName).foreach { brokerId =>
@@ -696,7 +696,7 @@ object ConfigCommand extends Logging {
696696
case t => t
697697
}
698698
sanitizedName match {
699-
case Some(ConfigEntityName.Default) => "default " + typeName
699+
case Some(ConfigEntityName.DEFAULT) => "default " + typeName
700700
case Some(n) =>
701701
val desanitized = if (entityType == ConfigType.USER || entityType == ConfigType.CLIENT) Sanitizer.desanitize(n) else n
702702
s"$typeName '$desanitized'"
@@ -757,7 +757,7 @@ object ConfigCommand extends Logging {
757757
else {
758758
// Exactly one entity type and at-most one entity name expected for other entities
759759
val name = entityNames.headOption match {
760-
case Some("") => Some(ConfigEntityName.Default)
760+
case Some("") => Some(ConfigEntityName.DEFAULT)
761761
case v => v
762762
}
763763
ConfigEntity(Entity(entityTypes.head, name), None)
@@ -774,7 +774,7 @@ object ConfigCommand extends Logging {
774774

775775
def sanitizeName(entityType: String, name: String) = {
776776
if (name.isEmpty)
777-
ConfigEntityName.Default
777+
ConfigEntityName.DEFAULT
778778
else {
779779
entityType match {
780780
case ConfigType.USER | ConfigType.CLIENT => Sanitizer.sanitize(name)

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

+6-5
Original file line numberDiff line numberDiff line change
@@ -30,6 +30,7 @@ import org.apache.kafka.common.metrics.Metrics
3030
import org.apache.kafka.common.metrics.stats.{Avg, CumulativeSum, Rate}
3131
import org.apache.kafka.common.security.auth.KafkaPrincipal
3232
import org.apache.kafka.common.utils.{Sanitizer, Time}
33+
import org.apache.kafka.server.config.ConfigEntityName
3334
import org.apache.kafka.server.quota.{ClientQuotaCallback, ClientQuotaEntity, ClientQuotaType}
3435
import org.apache.kafka.server.util.ShutdownableThread
3536

@@ -92,13 +93,13 @@ object ClientQuotaManager {
9293

9394
case object DefaultUserEntity extends BaseUserEntity {
9495
override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_USER
95-
override def name: String = ConfigEntityName.Default
96+
override def name: String = ConfigEntityName.DEFAULT
9697
override def toString: String = "default user"
9798
}
9899

99100
case object DefaultClientIdEntity extends ClientQuotaEntity.ConfigEntity {
100101
override def entityType: ClientQuotaEntity.ConfigEntityType = ClientQuotaEntity.ConfigEntityType.DEFAULT_CLIENT_ID
101-
override def name: String = ConfigEntityName.Default
102+
override def name: String = ConfigEntityName.DEFAULT
102103
override def toString: String = "default client-id"
103104
}
104105

@@ -109,7 +110,7 @@ object ClientQuotaManager {
109110

110111
def sanitizedUser: String = userEntity.map {
111112
case entity: UserEntity => entity.sanitizedUser
112-
case DefaultUserEntity => ConfigEntityName.Default
113+
case DefaultUserEntity => ConfigEntityName.DEFAULT
113114
}.getOrElse("")
114115

115116
def clientId: String = clientIdEntity.map(_.name).getOrElse("")
@@ -435,11 +436,11 @@ class ClientQuotaManager(private val config: ClientQuotaManagerConfig,
435436
lock.writeLock().lock()
436437
try {
437438
val userEntity = sanitizedUser.map {
438-
case ConfigEntityName.Default => DefaultUserEntity
439+
case ConfigEntityName.DEFAULT => DefaultUserEntity
439440
case user => UserEntity(user)
440441
}
441442
val clientIdEntity = sanitizedClientId.map {
442-
case ConfigEntityName.Default => DefaultClientIdEntity
443+
case ConfigEntityName.DEFAULT => DefaultClientIdEntity
443444
case _ => ClientIdEntity(clientId.getOrElse(throw new IllegalStateException("Client-id not provided")))
444445
}
445446
val quotaEntity = KafkaQuotaEntity(userEntity, clientIdEntity)

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

+4-3
Original file line numberDiff line numberDiff line change
@@ -34,6 +34,7 @@ import org.apache.kafka.common.metrics.Quota
3434
import org.apache.kafka.common.metrics.Quota._
3535
import org.apache.kafka.common.utils.Sanitizer
3636
import org.apache.kafka.server.ClientMetricsManager
37+
import org.apache.kafka.server.config.ConfigEntityName
3738
import org.apache.kafka.storage.internals.log.{LogConfig, ThrottledReplicaListValidator}
3839
import org.apache.kafka.storage.internals.log.LogConfig.MessageFormatVersion
3940

@@ -207,7 +208,7 @@ class UserConfigHandler(private val quotaManagers: QuotaManagers, val credential
207208
val sanitizedUser = entities(0)
208209
val sanitizedClientId = if (entities.length == 3) Some(entities(2)) else None
209210
updateQuotaConfig(Some(sanitizedUser), sanitizedClientId, config)
210-
if (sanitizedClientId.isEmpty && sanitizedUser != ConfigEntityName.Default)
211+
if (sanitizedClientId.isEmpty && sanitizedUser != ConfigEntityName.DEFAULT)
211212
credentialProvider.updateCredentials(Sanitizer.desanitize(sanitizedUser), config)
212213
}
213214
}
@@ -217,7 +218,7 @@ class IpConfigHandler(private val connectionQuotas: ConnectionQuotas) extends Co
217218
def processConfigChanges(ip: String, config: Properties): Unit = {
218219
val ipConnectionRateQuota = Option(config.getProperty(QuotaConfigs.IP_CONNECTION_RATE_OVERRIDE_CONFIG)).map(_.toInt)
219220
val updatedIp = {
220-
if (ip != ConfigEntityName.Default) {
221+
if (ip != ConfigEntityName.DEFAULT) {
221222
try {
222223
Some(InetAddress.getByName(ip))
223224
} catch {
@@ -245,7 +246,7 @@ class BrokerConfigHandler(private val brokerConfig: KafkaConfig,
245246
else
246247
DefaultReplicationThrottledRate
247248
}
248-
if (brokerId == ConfigEntityName.Default)
249+
if (brokerId == ConfigEntityName.DEFAULT)
249250
brokerConfig.dynamicConfig.updateDefaultConfig(properties)
250251
else if (brokerConfig.brokerId == brokerId.trim.toInt) {
251252
brokerConfig.dynamicConfig.updateBrokerConfig(brokerConfig.brokerId, properties)

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

+2-2
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,7 @@ import org.apache.kafka.common.config.types.Password
3636
import org.apache.kafka.common.network.{ListenerName, ListenerReconfigurable}
3737
import org.apache.kafka.common.security.authenticator.LoginManager
3838
import org.apache.kafka.common.utils.{ConfigUtils, Utils}
39-
import org.apache.kafka.server.config.{ConfigType, ServerTopicConfigSynonyms}
39+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType, ServerTopicConfigSynonyms}
4040
import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig
4141
import org.apache.kafka.server.metrics.ClientMetricsReceiverPlugin
4242
import org.apache.kafka.server.telemetry.ClientTelemetry
@@ -232,7 +232,7 @@ class DynamicBrokerConfig(private val kafkaConfig: KafkaConfig) extends Logging
232232

233233
zkClientOpt.foreach { zkClient =>
234234
val adminZkClient = new AdminZkClient(zkClient)
235-
updateDefaultConfig(adminZkClient.fetchEntityConfig(ConfigType.BROKER, ConfigEntityName.Default), false)
235+
updateDefaultConfig(adminZkClient.fetchEntityConfig(ConfigType.BROKER, ConfigEntityName.DEFAULT), false)
236236
val props = adminZkClient.fetchEntityConfig(ConfigType.BROKER, kafkaConfig.brokerId.toString)
237237
val brokerConfig = maybeReEncodePasswords(props, adminZkClient)
238238
updateBrokerConfig(kafkaConfig.brokerId, brokerConfig)

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

+2-1
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import org.apache.kafka.common.config.ConfigDef
2323
import org.apache.kafka.common.config.ConfigDef.Importance._
2424
import org.apache.kafka.common.config.ConfigDef.Range._
2525
import org.apache.kafka.common.config.ConfigDef.Type._
26+
import org.apache.kafka.server.config.ConfigEntityName
2627
import org.apache.kafka.storage.internals.log.LogConfig
2728

2829
import java.util
@@ -101,7 +102,7 @@ object DynamicConfig {
101102
def validate(props: Properties) = DynamicConfig.validate(ipConfigs, props, customPropsAllowed = false)
102103

103104
def isValidIpEntity(ip: String): Boolean = {
104-
if (ip != ConfigEntityName.Default) {
105+
if (ip != ConfigEntityName.DEFAULT) {
105106
try {
106107
InetAddress.getByName(ip)
107108
} catch {

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

+4-4
Original file line numberDiff line numberDiff line change
@@ -48,7 +48,7 @@ import org.apache.kafka.common.requests.{AlterConfigsRequest, ApiError}
4848
import org.apache.kafka.common.security.scram.internals.{ScramCredentialUtils, ScramFormatter}
4949
import org.apache.kafka.common.utils.Sanitizer
5050
import org.apache.kafka.server.common.AdminOperationException
51-
import org.apache.kafka.server.config.ConfigType
51+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType}
5252
import org.apache.kafka.storage.internals.log.LogConfig
5353

5454
import scala.collection.{Map, mutable, _}
@@ -518,7 +518,7 @@ class ZkAdminManager(val config: KafkaConfig,
518518
val perBrokerConfig = brokerId.nonEmpty
519519

520520
val persistentProps = if (perBrokerConfig) adminZkClient.fetchEntityConfig(ConfigType.BROKER, brokerId.get.toString)
521-
else adminZkClient.fetchEntityConfig(ConfigType.BROKER, ConfigEntityName.Default)
521+
else adminZkClient.fetchEntityConfig(ConfigType.BROKER, ConfigEntityName.DEFAULT)
522522

523523
val configProps = this.config.dynamicConfig.fromPersistentProps(persistentProps, perBrokerConfig)
524524
prepareIncrementalConfigs(alterConfigOps, configProps, KafkaConfig.configKeys)
@@ -559,13 +559,13 @@ class ZkAdminManager(val config: KafkaConfig,
559559

560560
private def sanitizeEntityName(entityName: String): String =
561561
Option(entityName) match {
562-
case None => ConfigEntityName.Default
562+
case None => ConfigEntityName.DEFAULT
563563
case Some(name) => Sanitizer.sanitize(name)
564564
}
565565

566566
private def desanitizeEntityName(sanitizedEntityName: String): String =
567567
sanitizedEntityName match {
568-
case ConfigEntityName.Default => null
568+
case ConfigEntityName.DEFAULT => null
569569
case name => Sanitizer.desanitize(name)
570570
}
571571

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

-4
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,6 @@ import org.apache.kafka.server.config.ConfigType
2929
import scala.jdk.CollectionConverters._
3030
import scala.collection._
3131

32-
object ConfigEntityName {
33-
val Default = "<default>"
34-
}
35-
3632
/**
3733
* This class initiates and carries out config changes for all entities defined in ConfigType.
3834
*

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

+6-6
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,6 @@
1818
package kafka.server.metadata
1919

2020
import kafka.network.ConnectionQuotas
21-
import kafka.server.ConfigEntityName
2221
import kafka.server.QuotaFactory.QuotaManagers
2322
import kafka.utils.Logging
2423
import org.apache.kafka.common.config.internals.QuotaConfigs
@@ -28,6 +27,7 @@ import org.apache.kafka.common.utils.Sanitizer
2827
import java.net.{InetAddress, UnknownHostException}
2928

3029
import org.apache.kafka.image.{ClientQuotaDelta, ClientQuotasDelta}
30+
import org.apache.kafka.server.config.ConfigEntityName
3131

3232
import scala.compat.java8.OptionConverters._
3333

@@ -147,13 +147,13 @@ class ClientQuotaMetadataManager(private[metadata] val quotaManagers: QuotaManag
147147
// Convert entity into Options with sanitized values for QuotaManagers
148148
val (sanitizedUser, sanitizedClientId) = quotaEntity match {
149149
case UserEntity(user) => (Some(Sanitizer.sanitize(user)), None)
150-
case DefaultUserEntity => (Some(ConfigEntityName.Default), None)
150+
case DefaultUserEntity => (Some(ConfigEntityName.DEFAULT), None)
151151
case ClientIdEntity(clientId) => (None, Some(Sanitizer.sanitize(clientId)))
152-
case DefaultClientIdEntity => (None, Some(ConfigEntityName.Default))
152+
case DefaultClientIdEntity => (None, Some(ConfigEntityName.DEFAULT))
153153
case ExplicitUserExplicitClientIdEntity(user, clientId) => (Some(Sanitizer.sanitize(user)), Some(Sanitizer.sanitize(clientId)))
154-
case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(ConfigEntityName.Default))
155-
case DefaultUserExplicitClientIdEntity(clientId) => (Some(ConfigEntityName.Default), Some(Sanitizer.sanitize(clientId)))
156-
case DefaultUserDefaultClientIdEntity => (Some(ConfigEntityName.Default), Some(ConfigEntityName.Default))
154+
case ExplicitUserDefaultClientIdEntity(user) => (Some(Sanitizer.sanitize(user)), Some(ConfigEntityName.DEFAULT))
155+
case DefaultUserExplicitClientIdEntity(clientId) => (Some(ConfigEntityName.DEFAULT), Some(Sanitizer.sanitize(clientId)))
156+
case DefaultUserDefaultClientIdEntity => (Some(ConfigEntityName.DEFAULT), Some(ConfigEntityName.DEFAULT))
157157
case IpEntity(_) | DefaultIpEntity => throw new IllegalStateException("Should not see IP quota entities here")
158158
}
159159

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

+3-3
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,12 @@ package kafka.server.metadata
1919

2020
import java.util.Properties
2121
import kafka.server.ConfigAdminManager.toLoggableProps
22-
import kafka.server.{ConfigEntityName, ConfigHandler, KafkaConfig}
22+
import kafka.server.{ConfigHandler, KafkaConfig}
2323
import kafka.utils.Logging
2424
import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, CLIENT_METRICS, 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
27+
import org.apache.kafka.server.config.{ConfigEntityName, 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(ConfigEntityName.Default, props)
81+
nodeConfigHandler.processConfigChanges(ConfigEntityName.DEFAULT, props)
8282
} catch {
8383
case t: Throwable => faultHandler.handleFault("Error updating " +
8484
s"cluster with new configuration: ${toLoggableProps(resource, props).mkString(",")} " +

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

+2-3
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,11 @@
1818
package kafka.server.metadata
1919

2020
import java.util.Properties
21-
import kafka.server.ConfigEntityName
2221
import kafka.zk.{AdminZkClient, KafkaZkClient}
2322
import org.apache.kafka.common.config.ConfigResource
2423
import org.apache.kafka.common.config.ConfigResource.Type
2524
import org.apache.kafka.common.errors.InvalidRequestException
26-
import org.apache.kafka.server.config.ConfigType
25+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType}
2726

2827

2928
object ZkConfigRepository {
@@ -42,7 +41,7 @@ class ZkConfigRepository(adminZkClient: AdminZkClient) extends ConfigRepository
4241
// ZK stores cluster configs under "<default>".
4342
val effectiveName = if (configResource.`type`.equals(Type.BROKER) &&
4443
configResource.name.isEmpty) {
45-
ConfigEntityName.Default
44+
ConfigEntityName.DEFAULT
4645
} else {
4746
configResource.name
4847
}

core/src/main/scala/kafka/zk/AdminZkClient.scala

+5-5
Original file line numberDiff line numberDiff line change
@@ -20,15 +20,15 @@ import java.util.{Collections, Optional, Properties}
2020
import kafka.admin.RackAwareMode
2121
import kafka.common.TopicAlreadyMarkedForDeletionException
2222
import kafka.controller.ReplicaAssignment
23-
import kafka.server.{ConfigEntityName, DynamicConfig, KafkaConfig}
23+
import kafka.server.{DynamicConfig, KafkaConfig}
2424
import kafka.utils._
2525
import kafka.utils.Implicits._
2626
import org.apache.kafka.admin.{AdminUtils, BrokerMetadata}
2727
import org.apache.kafka.common.{TopicPartition, Uuid}
2828
import org.apache.kafka.common.errors._
2929
import org.apache.kafka.common.internals.Topic
3030
import org.apache.kafka.server.common.AdminOperationException
31-
import org.apache.kafka.server.config.ConfigType
31+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType}
3232
import org.apache.kafka.storage.internals.log.LogConfig
3333
import org.apache.zookeeper.KeeperException.NodeExistsException
3434

@@ -345,7 +345,7 @@ class AdminZkClient(zkClient: KafkaZkClient,
345345
*/
346346
def parseBroker(broker: String): Option[Int] = {
347347
broker match {
348-
case ConfigEntityName.Default => None
348+
case ConfigEntityName.DEFAULT => None
349349
case _ =>
350350
try Some(broker.toInt)
351351
catch {
@@ -440,7 +440,7 @@ class AdminZkClient(zkClient: KafkaZkClient,
440440
*
441441
*/
442442
def changeUserOrUserClientIdConfig(sanitizedEntityName: String, configs: Properties, isUserClientId: Boolean = false): Unit = {
443-
if (sanitizedEntityName == ConfigEntityName.Default || sanitizedEntityName.contains("/clients"))
443+
if (sanitizedEntityName == ConfigEntityName.DEFAULT || sanitizedEntityName.contains("/clients"))
444444
DynamicConfig.Client.validate(configs)
445445
else
446446
DynamicConfig.User.validate(configs)
@@ -520,7 +520,7 @@ class AdminZkClient(zkClient: KafkaZkClient,
520520
*/
521521
def changeBrokerConfig(broker: Option[Int], configs: Properties): Unit = {
522522
validateBrokerConfig(configs)
523-
changeEntityConfig(ConfigType.BROKER, broker.map(_.toString).getOrElse(ConfigEntityName.Default), configs)
523+
changeEntityConfig(ConfigType.BROKER, broker.map(_.toString).getOrElse(ConfigEntityName.DEFAULT), configs)
524524
}
525525

526526
/**

core/src/main/scala/kafka/zk/migration/ZkConfigMigrationClient.scala

+4-4
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,7 @@
1717

1818
package kafka.zk.migration
1919

20-
import kafka.server.{ConfigEntityName, DynamicBrokerConfig, DynamicConfig, ZkAdminManager}
20+
import kafka.server.{DynamicBrokerConfig, DynamicConfig, ZkAdminManager}
2121
import kafka.utils.{Logging, PasswordEncoder}
2222
import kafka.zk.ZkMigrationClient.{logAndRethrow, wrapZkException}
2323
import kafka.zk._
@@ -31,7 +31,7 @@ import org.apache.kafka.common.quota.ClientQuotaEntity
3131
import org.apache.kafka.common.security.scram.internals.ScramCredentialUtils
3232
import org.apache.kafka.metadata.migration.ConfigMigrationClient.ClientQuotaVisitor
3333
import org.apache.kafka.metadata.migration.{ConfigMigrationClient, MigrationClientException, ZkMigrationLeadershipState}
34-
import org.apache.kafka.server.config.ConfigType
34+
import org.apache.kafka.server.config.{ConfigEntityName, ConfigType}
3535
import org.apache.zookeeper.KeeperException.Code
3636
import org.apache.zookeeper.{CreateMode, KeeperException}
3737

@@ -55,7 +55,7 @@ class ZkConfigMigrationClient(
5555
* to the special KRaft string.
5656
*/
5757
private def fromZkEntityName(entityName: String): String = {
58-
if (entityName.equals(ConfigEntityName.Default)) {
58+
if (entityName.equals(ConfigEntityName.DEFAULT)) {
5959
""
6060
} else {
6161
entityName
@@ -64,7 +64,7 @@ class ZkConfigMigrationClient(
6464

6565
private def toZkEntityName(entityName: String): String = {
6666
if (entityName.isEmpty) {
67-
ConfigEntityName.Default
67+
ConfigEntityName.DEFAULT
6868
} else {
6969
entityName
7070
}

0 commit comments

Comments
 (0)