You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ya...@apache.org on 2022/04/02 14:33:27 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #2277] Inline kyuubi prefix in KyuubiConf

This is an automated email from the ASF dual-hosted git repository.

yao pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new 6a23151  [KYUUBI #2277] Inline kyuubi prefix in KyuubiConf
6a23151 is described below

commit 6a231519454c5d75aca2120de140c06febf7ae54
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Sat Apr 2 22:33:18 2022 +0800

    [KYUUBI #2277] Inline kyuubi prefix in KyuubiConf
    
    ### _Why are the changes needed?_
    
    Inline `kyuubi.` prefix in `KyuubiConf` to keep consistently with the document and make searching friendly.
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [x] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #2277 from pan3793/conf.
    
    Closes #2277
    
    cca77ccb [Cheng Pan] fixup
    9c38925a [Cheng Pan] Inline kyuubi prefix in KyuubiConf
    
    Authored-by: Cheng Pan <ch...@apache.org>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 .../org/apache/kyuubi/engine/trino/TrinoConf.scala |   4 +-
 .../org/apache/kyuubi/config/KyuubiConf.scala      | 327 +++++++++++----------
 .../apache/kyuubi/config/ConfigEntrySuite.scala    |   2 +-
 .../apache/kyuubi/ha/HighAvailabilityConf.scala    |  43 +--
 .../org/apache/kyuubi/metrics/MetricsConf.scala    |  16 +-
 .../apache/kyuubi/zookeeper/ZookeeperConf.scala    |  20 +-
 6 files changed, 209 insertions(+), 203 deletions(-)

diff --git a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoConf.scala b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoConf.scala
index 9441112..cceb6d6 100644
--- a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoConf.scala
+++ b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoConf.scala
@@ -27,14 +27,14 @@ object TrinoConf {
   private def buildConf(key: String): ConfigBuilder = KyuubiConf.buildConf(key)
 
   val DATA_PROCESSING_POOL_SIZE: ConfigEntry[Int] =
-    buildConf("trino.client.data.processing.pool.size")
+    buildConf("kyuubi.trino.client.data.processing.pool.size")
       .doc("The size of the thread pool used by the trino client to processing data")
       .version("1.5.0")
       .intConf
       .createWithDefault(3)
 
   val CLIENT_REQUEST_TIMEOUT: ConfigEntry[Long] =
-    buildConf("trino.client.request.timeout")
+    buildConf("kyuubi.trino.client.request.timeout")
       .doc("Timeout for Trino client request to trino cluster")
       .version("1.5.0")
       .timeConf
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index 969a25e..f9200e5 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -183,28 +183,28 @@ object KyuubiConf {
   }
 
   def buildConf(key: String): ConfigBuilder = {
-    ConfigBuilder("kyuubi." + key).onCreate(register)
+    ConfigBuilder(key).onCreate(register)
   }
 
-  val SERVER_PRINCIPAL: OptionalConfigEntry[String] = buildConf("kinit.principal")
+  val SERVER_PRINCIPAL: OptionalConfigEntry[String] = buildConf("kyuubi.kinit.principal")
     .doc("Name of the Kerberos principal.")
     .version("1.0.0")
     .stringConf
     .createOptional
 
-  val SERVER_KEYTAB: OptionalConfigEntry[String] = buildConf("kinit.keytab")
+  val SERVER_KEYTAB: OptionalConfigEntry[String] = buildConf("kyuubi.kinit.keytab")
     .doc("Location of Kyuubi server's keytab.")
     .version("1.0.0")
     .stringConf
     .createOptional
 
-  val SERVER_SPNEGO_KEYTAB: OptionalConfigEntry[String] = buildConf("spnego.keytab")
+  val SERVER_SPNEGO_KEYTAB: OptionalConfigEntry[String] = buildConf("kyuubi.spnego.keytab")
     .doc("Keytab file for SPNego principal")
     .version("1.6.0")
     .stringConf
     .createOptional
 
-  val SERVER_SPNEGO_PRINCIPAL: OptionalConfigEntry[String] = buildConf("spnego.principal")
+  val SERVER_SPNEGO_PRINCIPAL: OptionalConfigEntry[String] = buildConf("kyuubi.spnego.principal")
     .doc("SPNego service principal, typical value would look like HTTP/_HOST@EXAMPLE.COM." +
       " SPNego service principal would be used when restful Kerberos security is enabled." +
       " This needs to be set only if SPNEGO is to be used in authentication.")
@@ -212,34 +212,34 @@ object KyuubiConf {
     .stringConf
     .createOptional
 
-  val KINIT_INTERVAL: ConfigEntry[Long] = buildConf("kinit.interval")
+  val KINIT_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.kinit.interval")
     .doc("How often will Kyuubi server run `kinit -kt [keytab] [principal]` to renew the" +
       " local Kerberos credentials cache")
     .version("1.0.0")
     .timeConf
     .createWithDefaultString("PT1H")
 
-  val KINIT_MAX_ATTEMPTS: ConfigEntry[Int] = buildConf("kinit.max.attempts")
+  val KINIT_MAX_ATTEMPTS: ConfigEntry[Int] = buildConf("kyuubi.kinit.max.attempts")
     .doc("How many times will `kinit` process retry")
     .version("1.0.0")
     .intConf
     .createWithDefault(10)
 
-  val OPERATION_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("operation.idle.timeout")
+  val OPERATION_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.operation.idle.timeout")
     .doc("Operation will be closed when it's not accessed for this duration of time")
     .version("1.0.0")
     .timeConf
     .createWithDefault(Duration.ofHours(3).toMillis)
 
   val CREDENTIALS_RENEWAL_INTERVAL: ConfigEntry[Long] =
-    buildConf("credentials.renewal.interval")
+    buildConf("kyuubi.credentials.renewal.interval")
       .doc("How often Kyuubi renews one user's delegation tokens")
       .version("1.4.0")
       .timeConf
       .createWithDefault(Duration.ofHours(1).toMillis)
 
   val CREDENTIALS_RENEWAL_RETRY_WAIT: ConfigEntry[Long] =
-    buildConf("credentials.renewal.retry.wait")
+    buildConf("kyuubi.credentials.renewal.retry.wait")
       .doc("How long to wait before retrying to fetch new credentials after a failure.")
       .version("1.4.0")
       .timeConf
@@ -247,7 +247,7 @@ object KyuubiConf {
       .createWithDefault(Duration.ofMinutes(1).toMillis)
 
   val CREDENTIALS_UPDATE_WAIT_TIMEOUT: ConfigEntry[Long] =
-    buildConf("credentials.update.wait.timeout")
+    buildConf("kyuubi.credentials.update.wait.timeout")
       .doc("How long to wait until credentials are ready.")
       .version("1.5.0")
       .timeConf
@@ -255,7 +255,7 @@ object KyuubiConf {
       .createWithDefault(Duration.ofMinutes(1).toMillis)
 
   val CREDENTIALS_CHECK_INTERVAL: ConfigEntry[Long] =
-    buildConf("credentials.check.interval")
+    buildConf("kyuubi.credentials.check.interval")
       .doc("The interval to check the expiration of cached <user, CredentialsRef> pairs.")
       .version("1.6.0")
       .timeConf
@@ -263,7 +263,7 @@ object KyuubiConf {
       .createWithDefault(Duration.ofMinutes(5).toMillis)
 
   val CREDENTIALS_IDLE_TIMEOUT: ConfigEntry[Long] =
-    buildConf("credentials.idle.timeout")
+    buildConf("kyuubi.credentials.idle.timeout")
       .doc("inactive users' credentials will be expired after a configured timeout")
       .version("1.6.0")
       .timeConf
@@ -271,14 +271,14 @@ object KyuubiConf {
       .createWithDefault(Duration.ofHours(6).toMillis)
 
   val CREDENTIALS_HADOOP_FS_ENABLED: ConfigEntry[Boolean] =
-    buildConf("credentials.hadoopfs.enabled")
+    buildConf("kyuubi.credentials.hadoopfs.enabled")
       .doc("Whether to renew Hadoop filesystem delegation tokens")
       .version("1.4.0")
       .booleanConf
       .createWithDefault(true)
 
   val CREDENTIALS_HADOOP_FS_URIS: ConfigEntry[Seq[String]] =
-    buildConf("credentials.hadoopfs.uris")
+    buildConf("kyuubi.credentials.hadoopfs.uris")
       .doc("Extra Hadoop filesystem URIs for which to request delegation tokens. " +
         "The filesystem that hosts fs.defaultFS does not need to be listed here.")
       .version("1.4.0")
@@ -287,7 +287,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val CREDENTIALS_HIVE_ENABLED: ConfigEntry[Boolean] =
-    buildConf("credentials.hive.enabled")
+    buildConf("kyuubi.credentials.hive.enabled")
       .doc("Whether to renew Hive metastore delegation token")
       .version("1.4.0")
       .booleanConf
@@ -303,7 +303,7 @@ object KyuubiConf {
   }
 
   val FRONTEND_PROTOCOLS: ConfigEntry[Seq[String]] =
-    buildConf("frontend.protocols")
+    buildConf("kyuubi.frontend.protocols")
       .doc("A comma separated list for all frontend protocols " +
         "<ul>" +
         " <li>THRIFT_BINARY - HiveServer2 compatible thrift binary protocol.</li>" +
@@ -319,7 +319,7 @@ object KyuubiConf {
         s"the frontend protocol should be one or more of ${FrontendProtocols.values.mkString(",")}")
       .createWithDefault(Seq(FrontendProtocols.THRIFT_BINARY.toString))
 
-  val FRONTEND_BIND_HOST: OptionalConfigEntry[String] = buildConf("frontend.bind.host")
+  val FRONTEND_BIND_HOST: OptionalConfigEntry[String] = buildConf("kyuubi.frontend.bind.host")
     .doc("(deprecated) Hostname or IP of the machine on which to run the thrift frontend service " +
       "via binary protocol.")
     .version("1.0.0")
@@ -327,14 +327,14 @@ object KyuubiConf {
     .createOptional
 
   val FRONTEND_THRIFT_BINARY_BIND_HOST: ConfigEntry[Option[String]] =
-    buildConf("frontend.thrift.binary.bind.host")
+    buildConf("kyuubi.frontend.thrift.binary.bind.host")
       .doc("Hostname or IP of the machine on which to run the thrift frontend service " +
         "via binary protocol.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_BIND_HOST)
 
   @deprecated(s"using ${FRONTEND_THRIFT_BINARY_BIND_PORT.key} instead", "1.4.0")
-  val FRONTEND_BIND_PORT: ConfigEntry[Int] = buildConf("frontend.bind.port")
+  val FRONTEND_BIND_PORT: ConfigEntry[Int] = buildConf("kyuubi.frontend.bind.port")
     .doc("(deprecated) Port of the machine on which to run the thrift frontend service " +
       "via binary protocol.")
     .version("1.0.0")
@@ -343,95 +343,97 @@ object KyuubiConf {
     .createWithDefault(10009)
 
   val FRONTEND_THRIFT_BINARY_BIND_PORT: ConfigEntry[Int] =
-    buildConf("frontend.thrift.binary.bind.port")
+    buildConf("kyuubi.frontend.thrift.binary.bind.port")
       .doc("Port of the machine on which to run the thrift frontend service via binary protocol.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_BIND_PORT)
 
-  val FRONTEND_MIN_WORKER_THREADS: ConfigEntry[Int] = buildConf("frontend.min.worker.threads")
-    .doc("(deprecated) Minimum number of threads in the of frontend worker thread pool for " +
-      "the thrift frontend service")
-    .version("1.0.0")
-    .intConf
-    .createWithDefault(9)
+  val FRONTEND_MIN_WORKER_THREADS: ConfigEntry[Int] =
+    buildConf("kyuubi.frontend.min.worker.threads")
+      .doc("(deprecated) Minimum number of threads in the of frontend worker thread pool for " +
+        "the thrift frontend service")
+      .version("1.0.0")
+      .intConf
+      .createWithDefault(9)
 
   val FRONTEND_THRIFT_MIN_WORKER_THREADS: ConfigEntry[Int] =
-    buildConf("frontend.thrift.min.worker.threads")
+    buildConf("kyuubi.frontend.thrift.min.worker.threads")
       .doc("Minimum number of threads in the of frontend worker thread pool for the thrift " +
         "frontend service")
       .version("1.4.0")
       .fallbackConf(FRONTEND_MIN_WORKER_THREADS)
 
-  val FRONTEND_MAX_WORKER_THREADS: ConfigEntry[Int] = buildConf("frontend.max.worker.threads")
-    .doc("(deprecated) Maximum number of threads in the of frontend worker thread pool for " +
-      "the thrift frontend service")
-    .version("1.0.0")
-    .intConf
-    .createWithDefault(999)
+  val FRONTEND_MAX_WORKER_THREADS: ConfigEntry[Int] =
+    buildConf("kyuubi.frontend.max.worker.threads")
+      .doc("(deprecated) Maximum number of threads in the of frontend worker thread pool for " +
+        "the thrift frontend service")
+      .version("1.0.0")
+      .intConf
+      .createWithDefault(999)
 
   val FRONTEND_THRIFT_MAX_WORKER_THREADS: ConfigEntry[Int] =
-    buildConf("frontend.thrift.max.worker.threads")
+    buildConf("kyuubi.frontend.thrift.max.worker.threads")
       .doc("Maximum number of threads in the of frontend worker thread pool for the thrift " +
         "frontend service")
       .version("1.4.0")
       .fallbackConf(FRONTEND_MAX_WORKER_THREADS)
 
   val FRONTEND_WORKER_KEEPALIVE_TIME: ConfigEntry[Long] =
-    buildConf("frontend.worker.keepalive.time")
+    buildConf("kyuubi.frontend.worker.keepalive.time")
       .doc("(deprecated) Keep-alive time (in milliseconds) for an idle worker thread")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofSeconds(60).toMillis)
 
   val FRONTEND_THRIFT_WORKER_KEEPALIVE_TIME: ConfigEntry[Long] =
-    buildConf("frontend.thrift.worker.keepalive.time")
+    buildConf("kyuubi.frontend.thrift.worker.keepalive.time")
       .doc("Keep-alive time (in milliseconds) for an idle worker thread")
       .version("1.4.0")
       .fallbackConf(FRONTEND_WORKER_KEEPALIVE_TIME)
 
   @deprecated(s"using ${FRONTEND_THRIFT_MAX_MESSAGE_SIZE.key} instead", "1.4.0")
   val FRONTEND_MAX_MESSAGE_SIZE: ConfigEntry[Int] =
-    buildConf("frontend.max.message.size")
+    buildConf("kyuubi.frontend.max.message.size")
       .doc("(deprecated) Maximum message size in bytes a Kyuubi server will accept.")
       .version("1.0.0")
       .intConf
       .createWithDefault(104857600)
 
   val FRONTEND_THRIFT_MAX_MESSAGE_SIZE: ConfigEntry[Int] =
-    buildConf("frontend.thrift.max.message.size")
+    buildConf("kyuubi.frontend.thrift.max.message.size")
       .doc("Maximum message size in bytes a Kyuubi server will accept.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_MAX_MESSAGE_SIZE)
 
   @deprecated(s"using ${FRONTEND_THRIFT_LOGIN_TIMEOUT.key} instead", "1.4.0")
   val FRONTEND_LOGIN_TIMEOUT: ConfigEntry[Long] =
-    buildConf("frontend.login.timeout")
+    buildConf("kyuubi.frontend.login.timeout")
       .doc("(deprecated) Timeout for Thrift clients during login to the thrift frontend service.")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofSeconds(20).toMillis)
 
   val FRONTEND_THRIFT_LOGIN_TIMEOUT: ConfigEntry[Long] =
-    buildConf("frontend.thrift.login.timeout")
+    buildConf("kyuubi.frontend.thrift.login.timeout")
       .doc("Timeout for Thrift clients during login to the thrift frontend service.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_LOGIN_TIMEOUT)
 
   @deprecated(s"using ${FRONTEND_THRIFT_LOGIN_BACKOFF_SLOT_LENGTH.key} instead", "1.4.0")
   val FRONTEND_LOGIN_BACKOFF_SLOT_LENGTH: ConfigEntry[Long] =
-    buildConf("frontend.backoff.slot.length")
+    buildConf("kyuubi.frontend.backoff.slot.length")
       .doc("(deprecated) Time to back off during login to the thrift frontend service.")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofMillis(100).toMillis)
 
   val FRONTEND_THRIFT_LOGIN_BACKOFF_SLOT_LENGTH: ConfigEntry[Long] =
-    buildConf("frontend.thrift.backoff.slot.length")
+    buildConf("kyuubi.frontend.thrift.backoff.slot.length")
       .doc("Time to back off during login to the thrift frontend service.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_LOGIN_BACKOFF_SLOT_LENGTH)
 
-  val AUTHENTICATION_METHOD: ConfigEntry[Seq[String]] = buildConf("authentication")
+  val AUTHENTICATION_METHOD: ConfigEntry[Seq[String]] = buildConf("kyuubi.authentication")
     .doc("A comma separated list of client authentication types.<ul>" +
       " <li>NOSASL: raw transport.</li>" +
       " <li>NONE: no authentication check.</li>" +
@@ -453,35 +455,36 @@ object KyuubiConf {
     .createWithDefault(Seq(AuthTypes.NONE.toString))
 
   val AUTHENTICATION_CUSTOM_CLASS: OptionalConfigEntry[String] =
-    buildConf("authentication.custom.class")
+    buildConf("kyuubi.authentication.custom.class")
       .doc("User-defined authentication implementation of " +
         "org.apache.kyuubi.service.authentication.PasswdAuthenticationProvider")
       .version("1.3.0")
       .stringConf
       .createOptional
 
-  val AUTHENTICATION_LDAP_URL: OptionalConfigEntry[String] = buildConf("authentication.ldap.url")
-    .doc("SPACE character separated LDAP connection URL(s).")
-    .version("1.0.0")
-    .stringConf
-    .createOptional
+  val AUTHENTICATION_LDAP_URL: OptionalConfigEntry[String] =
+    buildConf("kyuubi.authentication.ldap.url")
+      .doc("SPACE character separated LDAP connection URL(s).")
+      .version("1.0.0")
+      .stringConf
+      .createOptional
 
   val AUTHENTICATION_LDAP_BASEDN: OptionalConfigEntry[String] =
-    buildConf("authentication.ldap.base.dn")
+    buildConf("kyuubi.authentication.ldap.base.dn")
       .doc("LDAP base DN.")
       .version("1.0.0")
       .stringConf
       .createOptional
 
   val AUTHENTICATION_LDAP_DOMAIN: OptionalConfigEntry[String] =
-    buildConf("authentication.ldap.domain")
+    buildConf("kyuubi.authentication.ldap.domain")
       .doc("LDAP domain.")
       .version("1.0.0")
       .stringConf
       .createOptional
 
   val AUTHENTICATION_LDAP_GUIDKEY: ConfigEntry[String] =
-    buildConf("authentication.ldap.guidKey")
+    buildConf("kyuubi.authentication.ldap.guidKey")
       .doc("LDAP attribute name whose values are unique in this LDAP server." +
         "For example:uid or cn.")
       .version("1.2.0")
@@ -489,34 +492,34 @@ object KyuubiConf {
       .createWithDefault("uid")
 
   val DELEGATION_KEY_UPDATE_INTERVAL: ConfigEntry[Long] =
-    buildConf("delegation.key.update.interval")
+    buildConf("kyuubi.delegation.key.update.interval")
       .doc("unused yet")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofDays(1).toMillis)
 
   val DELEGATION_TOKEN_MAX_LIFETIME: ConfigEntry[Long] =
-    buildConf("delegation.token.max.lifetime")
+    buildConf("kyuubi.delegation.token.max.lifetime")
       .doc("unused yet")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofDays(7).toMillis)
 
   val DELEGATION_TOKEN_GC_INTERVAL: ConfigEntry[Long] =
-    buildConf("delegation.token.gc.interval")
+    buildConf("kyuubi.delegation.token.gc.interval")
       .doc("unused yet")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofHours(1).toMillis)
 
   val DELEGATION_TOKEN_RENEW_INTERVAL: ConfigEntry[Long] =
-    buildConf("delegation.token.renew.interval")
+    buildConf("kyuubi.delegation.token.renew.interval")
       .doc("unused yet")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofDays(7).toMillis)
 
-  val SASL_QOP: ConfigEntry[String] = buildConf("authentication.sasl.qop")
+  val SASL_QOP: ConfigEntry[String] = buildConf("kyuubi.authentication.sasl.qop")
     .doc("Sasl QOP enable higher levels of protection for Kyuubi communication with clients.<ul>" +
       " <li>auth - authentication only (default)</li>" +
       " <li>auth-int - authentication plus integrity protection</li>" +
@@ -529,12 +532,12 @@ object KyuubiConf {
     .createWithDefault(SaslQOP.AUTH.toString)
 
   val FRONTEND_REST_BIND_HOST: ConfigEntry[Option[String]] =
-    buildConf("frontend.rest.bind.host")
+    buildConf("kyuubi.frontend.rest.bind.host")
       .doc("Hostname or IP of the machine on which to run the REST frontend service.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_BIND_HOST)
 
-  val FRONTEND_REST_BIND_PORT: ConfigEntry[Int] = buildConf("frontend.rest.bind.port")
+  val FRONTEND_REST_BIND_PORT: ConfigEntry[Int] = buildConf("kyuubi.frontend.rest.bind.port")
     .doc("Port of the machine on which to run the REST frontend service.")
     .version("1.4.0")
     .intConf
@@ -542,12 +545,12 @@ object KyuubiConf {
     .createWithDefault(10099)
 
   val FRONTEND_MYSQL_BIND_HOST: ConfigEntry[Option[String]] =
-    buildConf("frontend.mysql.bind.host")
+    buildConf("kyuubi.frontend.mysql.bind.host")
       .doc("Hostname or IP of the machine on which to run the MySQL frontend service.")
       .version("1.4.0")
       .fallbackConf(FRONTEND_BIND_HOST)
 
-  val FRONTEND_MYSQL_BIND_PORT: ConfigEntry[Int] = buildConf("frontend.mysql.bind.port")
+  val FRONTEND_MYSQL_BIND_PORT: ConfigEntry[Int] = buildConf("kyuubi.frontend.mysql.bind.port")
     .doc("Port of the machine on which to run the MySQL frontend service.")
     .version("1.4.0")
     .intConf
@@ -565,7 +568,7 @@ object KyuubiConf {
    */
   val MAX_NETTY_THREADS: Int = 8
   val FRONTEND_MYSQL_NETTY_WORKER_THREADS: OptionalConfigEntry[Int] =
-    buildConf("frontend.mysql.netty.worker.threads")
+    buildConf("kyuubi.frontend.mysql.netty.worker.threads")
       .doc("Number of thread in the netty worker event loop of MySQL frontend service. " +
         s"Use min(cpu_cores, $MAX_NETTY_THREADS) in default.")
       .version("1.4.0")
@@ -576,21 +579,21 @@ object KyuubiConf {
       .createOptional
 
   val FRONTEND_MYSQL_MIN_WORKER_THREADS: ConfigEntry[Int] =
-    buildConf("frontend.mysql.min.worker.threads")
+    buildConf("kyuubi.frontend.mysql.min.worker.threads")
       .doc("Minimum number of threads in the command execution thread pool for the MySQL " +
         "frontend service")
       .version("1.4.0")
       .fallbackConf(FRONTEND_MIN_WORKER_THREADS)
 
   val FRONTEND_MYSQL_MAX_WORKER_THREADS: ConfigEntry[Int] =
-    buildConf("frontend.mysql.max.worker.threads")
+    buildConf("kyuubi.frontend.mysql.max.worker.threads")
       .doc("Maximum number of threads in the command execution thread pool for the MySQL " +
         "frontend service")
       .version("1.4.0")
       .fallbackConf(FRONTEND_MAX_WORKER_THREADS)
 
   val FRONTEND_MYSQL_WORKER_KEEPALIVE_TIME: ConfigEntry[Long] =
-    buildConf("frontend.mysql.worker.keepalive.time")
+    buildConf("kyuubi.frontend.mysql.worker.keepalive.time")
       .doc("Time(ms) that an idle async thread of the command execution thread pool will wait" +
         " for a new task to arrive before terminating in MySQL frontend service")
       .version("1.4.0")
@@ -601,7 +604,7 @@ object KyuubiConf {
   // ///////////////////////////////////////////////////////////////////////////////////////////////
 
   val ENGINE_ERROR_MAX_SIZE: ConfigEntry[Int] =
-    buildConf("session.engine.startup.error.max.size")
+    buildConf("kyuubi.session.engine.startup.error.max.size")
       .doc("During engine bootstrapping, if error occurs, using this config to limit the length" +
         " error message(characters).")
       .version("1.1.0")
@@ -609,7 +612,7 @@ object KyuubiConf {
       .checkValue(v => v >= 200 && v <= 8192, s"must in [200, 8192]")
       .createWithDefault(8192)
 
-  val ENGINE_LOG_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.log.timeout")
+  val ENGINE_LOG_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.engine.log.timeout")
     .doc("If we use Spark as the engine then the session submit log is the console output of " +
       "spark-submit. We will retain the session submit log until over the config value.")
     .version("1.1.0")
@@ -618,7 +621,7 @@ object KyuubiConf {
     .createWithDefault(Duration.ofDays(1).toMillis)
 
   val ENGINE_SPARK_MAIN_RESOURCE: OptionalConfigEntry[String] =
-    buildConf("session.engine.spark.main.resource")
+    buildConf("kyuubi.session.engine.spark.main.resource")
       .doc("The package used to create Spark SQL engine remote application. If it is undefined," +
         " Kyuubi will use the default")
       .version("1.0.0")
@@ -626,7 +629,7 @@ object KyuubiConf {
       .createOptional
 
   val ENGINE_FLINK_MAIN_RESOURCE: OptionalConfigEntry[String] =
-    buildConf("session.engine.flink.main.resource")
+    buildConf("kyuubi.session.engine.flink.main.resource")
       .doc("The package used to create Flink SQL engine remote job. If it is undefined," +
         " Kyuubi will use the default")
       .version("1.4.0")
@@ -634,7 +637,7 @@ object KyuubiConf {
       .createOptional
 
   val ENGINE_FLINK_MAX_ROWS: ConfigEntry[Int] =
-    buildConf("session.engine.flink.max.rows")
+    buildConf("kyuubi.session.engine.flink.max.rows")
       .doc("Max rows of Flink query results. For batch queries, rows that exceeds the limit " +
         "would be ignored. For streaming queries, the query would be canceled if the limit " +
         "is reached.")
@@ -643,7 +646,7 @@ object KyuubiConf {
       .createWithDefault(1000000)
 
   val ENGINE_TRINO_MAIN_RESOURCE: OptionalConfigEntry[String] =
-    buildConf("session.engine.trino.main.resource")
+    buildConf("kyuubi.session.engine.trino.main.resource")
       .doc("The package used to create Trino engine remote job. If it is undefined," +
         " Kyuubi will use the default")
       .version("1.5.0")
@@ -651,41 +654,41 @@ object KyuubiConf {
       .createOptional
 
   val ENGINE_TRINO_CONNECTION_URL: OptionalConfigEntry[String] =
-    buildConf("session.engine.trino.connection.url")
+    buildConf("kyuubi.session.engine.trino.connection.url")
       .doc("The server url that trino engine will connect to")
       .version("1.5.0")
       .stringConf
       .createOptional
 
   val ENGINE_TRINO_CONNECTION_CATALOG: OptionalConfigEntry[String] =
-    buildConf("session.engine.trino.connection.catalog")
+    buildConf("kyuubi.session.engine.trino.connection.catalog")
       .doc("The default catalog that trino engine will connect to")
       .version("1.5.0")
       .stringConf
       .createOptional
 
   val ENGINE_HIVE_MAIN_RESOURCE: OptionalConfigEntry[String] =
-    buildConf("session.engine.hive.main.resource")
+    buildConf("kyuubi.session.engine.hive.main.resource")
       .doc("The package used to create Hive engine remote job. If it is undefined," +
         " Kyuubi will use the default")
       .version("1.6.0")
       .stringConf
       .createOptional
 
-  val ENGINE_LOGIN_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.login.timeout")
+  val ENGINE_LOGIN_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.engine.login.timeout")
     .doc("The timeout of creating the connection to remote sql query engine")
     .version("1.0.0")
     .timeConf
     .createWithDefault(Duration.ofSeconds(15).toMillis)
 
-  val ENGINE_REQUEST_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.request.timeout")
+  val ENGINE_REQUEST_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.engine.request.timeout")
     .doc("The timeout of awaiting response after sending request to remote sql query engine")
     .version("1.4.0")
     .timeConf
     .createWithDefault(Duration.ofSeconds(60).toMillis)
 
   val ENGINE_ALIVE_PROBE_ENABLED: ConfigEntry[Boolean] =
-    buildConf("session.engine.alive.probe.enabled")
+    buildConf("kyuubi.session.engine.alive.probe.enabled")
       .doc("Whether to enable the engine alive probe, it true, we will create a companion thrift" +
         " client that sends simple request to check whether the engine is keep alive.")
       .version("1.6.0")
@@ -693,27 +696,27 @@ object KyuubiConf {
       .createWithDefault(false)
 
   val ENGINE_ALIVE_PROBE_INTERVAL: ConfigEntry[Long] =
-    buildConf("session.engine.alive.probe.interval")
+    buildConf("kyuubi.session.engine.alive.probe.interval")
       .doc("The interval for engine alive probe.")
       .version("1.6.0")
       .timeConf
       .createWithDefault(Duration.ofSeconds(10).toMillis)
 
   val ENGINE_ALIVE_TIMEOUT: ConfigEntry[Long] =
-    buildConf("session.engine.alive.timeout")
+    buildConf("kyuubi.session.engine.alive.timeout")
       .doc("The timeout for engine alive. If there is no alive probe success in the last timeout" +
         " window, the engine will be marked as no-alive.")
       .version("1.6.0")
       .timeConf
       .createWithDefault(Duration.ofSeconds(120).toMillis)
 
-  val ENGINE_INIT_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.initialize.timeout")
+  val ENGINE_INIT_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.engine.initialize.timeout")
     .doc("Timeout for starting the background engine, e.g. SparkSQLEngine.")
     .version("1.0.0")
     .timeConf
     .createWithDefault(Duration.ofSeconds(180).toMillis)
 
-  val SESSION_CHECK_INTERVAL: ConfigEntry[Long] = buildConf("session.check.interval")
+  val SESSION_CHECK_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.session.check.interval")
     .doc("The check interval for session timeout.")
     .version("1.0.0")
     .timeConf
@@ -721,26 +724,26 @@ object KyuubiConf {
     .createWithDefault(Duration.ofMinutes(5).toMillis)
 
   @deprecated(s"using ${SESSION_IDLE_TIMEOUT.key} instead", "1.2.0")
-  val SESSION_TIMEOUT: ConfigEntry[Long] = buildConf("session.timeout")
+  val SESSION_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.timeout")
     .doc("(deprecated)session timeout, it will be closed when it's not accessed for this duration")
     .version("1.0.0")
     .timeConf
     .checkValue(_ >= Duration.ofSeconds(3).toMillis, "Minimum 3 seconds")
     .createWithDefault(Duration.ofHours(6).toMillis)
 
-  val SESSION_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("session.idle.timeout")
+  val SESSION_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.idle.timeout")
     .doc("session idle timeout, it will be closed when it's not accessed for this duration")
     .version("1.2.0")
     .fallbackConf(SESSION_TIMEOUT)
 
-  val ENGINE_CHECK_INTERVAL: ConfigEntry[Long] = buildConf("session.engine.check.interval")
+  val ENGINE_CHECK_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.session.engine.check.interval")
     .doc("The check interval for engine timeout")
     .version("1.0.0")
     .timeConf
     .checkValue(_ >= Duration.ofSeconds(1).toMillis, "Minimum 1 seconds")
     .createWithDefault(Duration.ofMinutes(1).toMillis)
 
-  val ENGINE_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("session.engine.idle.timeout")
+  val ENGINE_IDLE_TIMEOUT: ConfigEntry[Long] = buildConf("kyuubi.session.engine.idle.timeout")
     .doc("engine timeout, the engine will self-terminate when it's not accessed for this " +
       "duration. 0 or negative means not to self-terminate.")
     .version("1.0.0")
@@ -748,7 +751,7 @@ object KyuubiConf {
     .createWithDefault(Duration.ofMinutes(30L).toMillis)
 
   val SESSION_CONF_IGNORE_LIST: ConfigEntry[Seq[String]] =
-    buildConf("session.conf.ignore.list")
+    buildConf("kyuubi.session.conf.ignore.list")
       .doc("A comma separated list of ignored keys. If the client connection contains any of" +
         " them, the key and the corresponding value will be removed silently during engine" +
         " bootstrap and connection setup." +
@@ -761,7 +764,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val SESSION_CONF_RESTRICT_LIST: ConfigEntry[Seq[String]] =
-    buildConf("session.conf.restrict.list")
+    buildConf("kyuubi.session.conf.restrict.list")
       .doc("A comma separated list of restricted keys. If the client connection contains any of" +
         " them, the connection will be rejected explicitly during engine bootstrap and connection" +
         " setup." +
@@ -774,7 +777,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val SESSION_ENGINE_STARTUP_MAX_LOG_LINES: ConfigEntry[Int] =
-    buildConf("session.engine.startup.maxLogLines")
+    buildConf("kyuubi.session.engine.startup.maxLogLines")
       .doc("The maximum number of engine log lines when errors occur during engine startup phase." +
         " Note that this max lines is for client-side to help track engine startup issue.")
       .version("1.4.0")
@@ -783,7 +786,7 @@ object KyuubiConf {
       .createWithDefault(10)
 
   val SESSION_ENGINE_STARTUP_WAIT_COMPLETION: ConfigEntry[Boolean] =
-    buildConf("session.engine.startup.waitCompletion")
+    buildConf("kyuubi.session.engine.startup.waitCompletion")
       .doc("Whether to wait for completion after engine starts." +
         " If false, the startup process will be destroyed after the engine is started." +
         " Note that only use it when the driver is not running locally," +
@@ -793,7 +796,7 @@ object KyuubiConf {
       .createWithDefault(true)
 
   val SESSION_ENGINE_LAUNCH_ASYNC: ConfigEntry[Boolean] =
-    buildConf("session.engine.launch.async")
+    buildConf("kyuubi.session.engine.launch.async")
       .doc("When opening kyuubi session, whether to launch backend engine asynchronously." +
         " When true, the Kyuubi server will set up the connection with the client without delay" +
         " as the backend engine will be created asynchronously.")
@@ -802,34 +805,34 @@ object KyuubiConf {
       .createWithDefault(true)
 
   val SERVER_EXEC_POOL_SIZE: ConfigEntry[Int] =
-    buildConf("backend.server.exec.pool.size")
+    buildConf("kyuubi.backend.server.exec.pool.size")
       .doc("Number of threads in the operation execution thread pool of Kyuubi server")
       .version("1.0.0")
       .intConf
       .createWithDefault(100)
 
   val ENGINE_EXEC_POOL_SIZE: ConfigEntry[Int] =
-    buildConf("backend.engine.exec.pool.size")
+    buildConf("kyuubi.backend.engine.exec.pool.size")
       .doc("Number of threads in the operation execution thread pool of SQL engine applications")
       .version("1.0.0")
       .fallbackConf(SERVER_EXEC_POOL_SIZE)
 
   val SERVER_EXEC_WAIT_QUEUE_SIZE: ConfigEntry[Int] =
-    buildConf("backend.server.exec.pool.wait.queue.size")
+    buildConf("kyuubi.backend.server.exec.pool.wait.queue.size")
       .doc("Size of the wait queue for the operation execution thread pool of Kyuubi server")
       .version("1.0.0")
       .intConf
       .createWithDefault(100)
 
   val ENGINE_EXEC_WAIT_QUEUE_SIZE: ConfigEntry[Int] =
-    buildConf("backend.engine.exec.pool.wait.queue.size")
+    buildConf("kyuubi.backend.engine.exec.pool.wait.queue.size")
       .doc("Size of the wait queue for the operation execution thread pool in SQL engine" +
         " applications")
       .version("1.0.0")
       .fallbackConf(SERVER_EXEC_WAIT_QUEUE_SIZE)
 
   val SERVER_EXEC_KEEPALIVE_TIME: ConfigEntry[Long] =
-    buildConf("backend.server.exec.pool.keepalive.time")
+    buildConf("kyuubi.backend.server.exec.pool.keepalive.time")
       .doc("Time(ms) that an idle async thread of the operation execution thread pool will wait" +
         " for a new task to arrive before terminating in Kyuubi server")
       .version("1.0.0")
@@ -837,28 +840,28 @@ object KyuubiConf {
       .createWithDefault(Duration.ofSeconds(60).toMillis)
 
   val ENGINE_EXEC_KEEPALIVE_TIME: ConfigEntry[Long] =
-    buildConf("backend.engine.exec.pool.keepalive.time")
+    buildConf("kyuubi.backend.engine.exec.pool.keepalive.time")
       .doc("Time(ms) that an idle async thread of the operation execution thread pool will wait" +
         " for a new task to arrive before terminating in SQL engine applications")
       .version("1.0.0")
       .fallbackConf(SERVER_EXEC_KEEPALIVE_TIME)
 
   val SERVER_EXEC_POOL_SHUTDOWN_TIMEOUT: ConfigEntry[Long] =
-    buildConf("backend.server.exec.pool.shutdown.timeout")
+    buildConf("kyuubi.backend.server.exec.pool.shutdown.timeout")
       .doc("Timeout(ms) for the operation execution thread pool to terminate in Kyuubi server")
       .version("1.0.0")
       .timeConf
       .createWithDefault(Duration.ofSeconds(10).toMillis)
 
   val ENGINE_EXEC_POOL_SHUTDOWN_TIMEOUT: ConfigEntry[Long] =
-    buildConf("backend.engine.exec.pool.shutdown.timeout")
+    buildConf("kyuubi.backend.engine.exec.pool.shutdown.timeout")
       .doc("Timeout(ms) for the operation execution thread pool to terminate in SQL engine" +
         " applications")
       .version("1.0.0")
       .fallbackConf(SERVER_EXEC_POOL_SHUTDOWN_TIMEOUT)
 
   val OPERATION_STATUS_POLLING_TIMEOUT: ConfigEntry[Long] =
-    buildConf("operation.status.polling.timeout")
+    buildConf("kyuubi.operation.status.polling.timeout")
       .doc("Timeout(ms) for long polling asynchronous running sql query's status")
       .version("1.0.0")
       .timeConf
@@ -866,14 +869,14 @@ object KyuubiConf {
 
   @deprecated(s"using kyuubi.operation.thrift.client.request.max.attempts instead", "1.6.0")
   val OPERATION_STATUS_POLLING_MAX_ATTEMPTS: ConfigEntry[Int] =
-    buildConf("operation.status.polling.max.attempts")
+    buildConf("kyuubi.operation.status.polling.max.attempts")
       .doc(s"(deprecated) - Using kyuubi.operation.thrift.client.request.max.attempts instead")
       .version("1.4.0")
       .intConf
       .createWithDefault(5)
 
   val OPERATION_THRIFT_CLIENT_REQUEST_MAX_ATTEMPTS: ConfigEntry[Int] =
-    buildConf("operation.thrift.client.request.max.attempts")
+    buildConf("kyuubi.operation.thrift.client.request.max.attempts")
       .doc("Max attempts for operation thrift request call at server-side on raw transport" +
         " failures, e.g. TTransportException")
       .version("1.6.0")
@@ -882,7 +885,7 @@ object KyuubiConf {
       .createWithDefault(5)
 
   val OPERATION_FORCE_CANCEL: ConfigEntry[Boolean] =
-    buildConf("operation.interrupt.on.cancel")
+    buildConf("kyuubi.operation.interrupt.on.cancel")
       .doc("When true, all running tasks will be interrupted if one cancels a query. " +
         "When false, all running tasks will remain until finished.")
       .version("1.2.0")
@@ -890,7 +893,7 @@ object KyuubiConf {
       .createWithDefault(true)
 
   val OPERATION_QUERY_TIMEOUT: OptionalConfigEntry[Long] =
-    buildConf("operation.query.timeout")
+    buildConf("kyuubi.operation.query.timeout")
       .doc("Timeout for query executions at server-side, take affect with client-side timeout(" +
         "`java.sql.Statement.setQueryTimeout`) together, a running query will be cancelled" +
         " automatically if timeout. It's off by default, which means only client-side take fully" +
@@ -903,7 +906,7 @@ object KyuubiConf {
       .createOptional
 
   val OPERATION_INCREMENTAL_COLLECT: ConfigEntry[Boolean] =
-    buildConf("operation.incremental.collect")
+    buildConf("kyuubi.operation.incremental.collect")
       .internal
       .doc("When true, the executor side result will be sequentially calculated and returned to" +
         " the Spark driver side.")
@@ -912,7 +915,7 @@ object KyuubiConf {
       .createWithDefault(false)
 
   val OPERATION_RESULT_MAX_ROWS: ConfigEntry[Int] =
-    buildConf("operation.result.max.rows")
+    buildConf("kyuubi.operation.result.max.rows")
       .doc("Max rows of Spark query results. Rows that exceeds the limit would be ignored. " +
         "By setting this value to 0 to disable the max rows limit.")
       .version("1.6.0")
@@ -920,20 +923,21 @@ object KyuubiConf {
       .createWithDefault(0)
 
   val SERVER_OPERATION_LOG_DIR_ROOT: ConfigEntry[String] =
-    buildConf("operation.log.dir.root")
+    buildConf("kyuubi.operation.log.dir.root")
       .doc("Root directory for query operation log at server-side.")
       .version("1.4.0")
       .stringConf
       .createWithDefault("server_operation_logs")
 
   @deprecated(s"using kyuubi.engine.share.level instead", "1.2.0")
-  val LEGACY_ENGINE_SHARE_LEVEL: ConfigEntry[String] = buildConf("session.engine.share.level")
-    .doc(s"(deprecated) - Using kyuubi.engine.share.level instead")
-    .version("1.0.0")
-    .stringConf
-    .transform(_.toUpperCase(Locale.ROOT))
-    .checkValues(ShareLevel.values.map(_.toString))
-    .createWithDefault(ShareLevel.USER.toString)
+  val LEGACY_ENGINE_SHARE_LEVEL: ConfigEntry[String] =
+    buildConf("kyuubi.session.engine.share.level")
+      .doc(s"(deprecated) - Using kyuubi.engine.share.level instead")
+      .version("1.0.0")
+      .stringConf
+      .transform(_.toUpperCase(Locale.ROOT))
+      .checkValues(ShareLevel.values.map(_.toString))
+      .createWithDefault(ShareLevel.USER.toString)
 
   // [ZooKeeper Data Model]
   // (http://zookeeper.apache.org/doc/r3.7.0/zookeeperProgrammers.html#ch_zkDataModel)
@@ -942,7 +946,7 @@ object KyuubiConf {
 
   @deprecated(s"using kyuubi.engine.share.level.subdomain instead", "1.4.0")
   val ENGINE_SHARE_LEVEL_SUB_DOMAIN: ConfigEntry[Option[String]] =
-    buildConf("engine.share.level.sub.domain")
+    buildConf("kyuubi.engine.share.level.sub.domain")
       .doc("(deprecated) - Using kyuubi.engine.share.level.subdomain instead")
       .version("1.2.0")
       .stringConf
@@ -951,7 +955,7 @@ object KyuubiConf {
       .createOptional
 
   val ENGINE_SHARE_LEVEL_SUBDOMAIN: ConfigEntry[Option[String]] =
-    buildConf("engine.share.level.subdomain")
+    buildConf("kyuubi.engine.share.level.subdomain")
       .doc("Allow end-users to create a subdomain for the share level of an engine. A" +
         " subdomain is a case-insensitive string values that must be a valid zookeeper sub path." +
         " For example, for `USER` share level, an end-user can share a certain engine within" +
@@ -962,7 +966,7 @@ object KyuubiConf {
 
   @deprecated(s"using ${FRONTEND_CONNECTION_URL_USE_HOSTNAME.key} instead, 1.5.0")
   val ENGINE_CONNECTION_URL_USE_HOSTNAME: ConfigEntry[Boolean] =
-    buildConf("engine.connection.url.use.hostname")
+    buildConf("kyuubi.engine.connection.url.use.hostname")
       .doc("(deprecated) " +
         "When true, engine register with hostname to zookeeper. When spark run on k8s" +
         " with cluster mode, set to false to ensure that server can connect to engine")
@@ -971,12 +975,12 @@ object KyuubiConf {
       .createWithDefault(true)
 
   val FRONTEND_CONNECTION_URL_USE_HOSTNAME: ConfigEntry[Boolean] =
-    buildConf("frontend.connection.url.use.hostname")
+    buildConf("kyuubi.frontend.connection.url.use.hostname")
       .doc("When true, frontend services prefer hostname, otherwise, ip address")
       .version("1.5.0")
       .fallbackConf(ENGINE_CONNECTION_URL_USE_HOSTNAME)
 
-  val ENGINE_SHARE_LEVEL: ConfigEntry[String] = buildConf("engine.share.level")
+  val ENGINE_SHARE_LEVEL: ConfigEntry[String] = buildConf("kyuubi.engine.share.level")
     .doc("Engines will be shared in different levels, available configs are: <ul>" +
       " <li>CONNECTION: engine will not be shared but only used by the current client" +
       " connection</li>" +
@@ -992,7 +996,7 @@ object KyuubiConf {
     .version("1.2.0")
     .fallbackConf(LEGACY_ENGINE_SHARE_LEVEL)
 
-  val ENGINE_TYPE: ConfigEntry[String] = buildConf("engine.type")
+  val ENGINE_TYPE: ConfigEntry[String] = buildConf("kyuubi.engine.type")
     .doc("Specify the detailed engine that supported by the Kyuubi. The engine type bindings to" +
       " SESSION scope. This configuration is experimental. Currently, available configs are: <ul>" +
       " <li>SPARK_SQL: specify this engine type will launch a Spark engine which can provide" +
@@ -1008,14 +1012,14 @@ object KyuubiConf {
     .checkValues(EngineType.values.map(_.toString))
     .createWithDefault(EngineType.SPARK_SQL.toString)
 
-  val ENGINE_POOL_NAME: ConfigEntry[String] = buildConf("engine.pool.name")
+  val ENGINE_POOL_NAME: ConfigEntry[String] = buildConf("kyuubi.engine.pool.name")
     .doc("The name of engine pool.")
     .version("1.5.0")
     .stringConf
     .checkValue(validZookeeperSubPath.matcher(_).matches(), "must be valid zookeeper sub path.")
     .createWithDefault("engine-pool")
 
-  val ENGINE_POOL_SIZE_THRESHOLD: ConfigEntry[Int] = buildConf("engine.pool.size.threshold")
+  val ENGINE_POOL_SIZE_THRESHOLD: ConfigEntry[Int] = buildConf("kyuubi.engine.pool.size.threshold")
     .doc("This parameter is introduced as a server-side parameter, " +
       "and controls the upper limit of the engine pool.")
     .version("1.4.0")
@@ -1023,7 +1027,7 @@ object KyuubiConf {
     .checkValue(s => s > 0 && s < 33, "Invalid engine pool threshold, it should be in [1, 32]")
     .createWithDefault(9)
 
-  val ENGINE_POOL_SIZE: ConfigEntry[Int] = buildConf("engine.pool.size")
+  val ENGINE_POOL_SIZE: ConfigEntry[Int] = buildConf("kyuubi.engine.pool.size")
     .doc("The size of engine pool. Note that, " +
       "if the size is less than 1, the engine pool will not be enabled; " +
       "otherwise, the size of the engine pool will be " +
@@ -1033,7 +1037,7 @@ object KyuubiConf {
     .createWithDefault(-1)
 
   val ENGINE_INITIALIZE_SQL: ConfigEntry[Seq[String]] =
-    buildConf("engine.initialize.sql")
+    buildConf("kyuubi.engine.initialize.sql")
       .doc("SemiColon-separated list of SQL statements to be initialized in the newly created " +
         "engine before queries. i.e. use `SHOW DATABASES` to eagerly active HiveClient. This " +
         "configuration can not be used in JDBC url due to the limitation of Beeline/JDBC driver.")
@@ -1043,7 +1047,7 @@ object KyuubiConf {
       .createWithDefaultString("SHOW DATABASES")
 
   val ENGINE_SESSION_INITIALIZE_SQL: ConfigEntry[Seq[String]] =
-    buildConf("engine.session.initialize.sql")
+    buildConf("kyuubi.engine.session.initialize.sql")
       .doc("SemiColon-separated list of SQL statements to be initialized in the newly created " +
         "engine session before queries. This configuration can not be used in JDBC url due to " +
         "the limitation of Beeline/JDBC driver.")
@@ -1053,7 +1057,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val ENGINE_DEREGISTER_EXCEPTION_CLASSES: ConfigEntry[Seq[String]] =
-    buildConf("engine.deregister.exception.classes")
+    buildConf("kyuubi.engine.deregister.exception.classes")
       .doc("A comma separated list of exception classes. If there is any exception thrown," +
         " whose class matches the specified classes, the engine would deregister itself.")
       .version("1.2.0")
@@ -1062,7 +1066,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val ENGINE_DEREGISTER_EXCEPTION_MESSAGES: ConfigEntry[Seq[String]] =
-    buildConf("engine.deregister.exception.messages")
+    buildConf("kyuubi.engine.deregister.exception.messages")
       .doc("A comma separated list of exception messages. If there is any exception thrown," +
         " whose message or stacktrace matches the specified message list, the engine would" +
         " deregister itself.")
@@ -1072,7 +1076,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val ENGINE_DEREGISTER_JOB_MAX_FAILURES: ConfigEntry[Int] =
-    buildConf("engine.deregister.job.max.failures")
+    buildConf("kyuubi.engine.deregister.job.max.failures")
       .doc("Number of failures of job before deregistering the engine.")
       .version("1.2.0")
       .intConf
@@ -1080,7 +1084,7 @@ object KyuubiConf {
       .createWithDefault(4)
 
   val ENGINE_DEREGISTER_EXCEPTION_TTL: ConfigEntry[Long] =
-    buildConf("engine.deregister.exception.ttl")
+    buildConf("kyuubi.engine.deregister.exception.ttl")
       .doc(s"Time to live(TTL) for exceptions pattern specified in" +
         s" ${ENGINE_DEREGISTER_EXCEPTION_CLASSES.key} and" +
         s" ${ENGINE_DEREGISTER_EXCEPTION_MESSAGES.key} to deregister engines. Once the total" +
@@ -1092,15 +1096,16 @@ object KyuubiConf {
       .checkValue(_ > 0, "must be positive number")
       .createWithDefault(Duration.ofMinutes(30).toMillis)
 
-  val OPERATION_SCHEDULER_POOL: OptionalConfigEntry[String] = buildConf("operation.scheduler.pool")
-    .doc("The scheduler pool of job. Note that, this config should be used after change Spark " +
-      "config spark.scheduler.mode=FAIR.")
-    .version("1.1.1")
-    .stringConf
-    .createOptional
+  val OPERATION_SCHEDULER_POOL: OptionalConfigEntry[String] =
+    buildConf("kyuubi.operation.scheduler.pool")
+      .doc("The scheduler pool of job. Note that, this config should be used after change Spark " +
+        "config spark.scheduler.mode=FAIR.")
+      .version("1.1.1")
+      .stringConf
+      .createOptional
 
   val ENGINE_SINGLE_SPARK_SESSION: ConfigEntry[Boolean] =
-    buildConf("engine.single.spark.session")
+    buildConf("kyuubi.engine.single.spark.session")
       .doc("When set to true, this engine is running in a single session mode. " +
         "All the JDBC/ODBC connections share the temporary views, function registries, " +
         "SQL configuration and the current database.")
@@ -1109,14 +1114,14 @@ object KyuubiConf {
       .createWithDefault(false)
 
   val SERVER_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
-    buildConf("backend.server.event.json.log.path")
+    buildConf("kyuubi.backend.server.event.json.log.path")
       .doc("The location of server events go for the builtin JSON logger")
       .version("1.4.0")
       .stringConf
       .createWithDefault("file:///tmp/kyuubi/events")
 
   val ENGINE_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
-    buildConf("engine.event.json.log.path")
+    buildConf("kyuubi.engine.event.json.log.path")
       .doc("The location of all the engine events go for the builtin JSON logger.<ul>" +
         "<li>Local Path: start with 'file://'</li>" +
         "<li>HDFS Path: start with 'hdfs://'</li></ul>")
@@ -1125,7 +1130,7 @@ object KyuubiConf {
       .createWithDefault("file:///tmp/kyuubi/events")
 
   val SERVER_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
-    buildConf("backend.server.event.loggers")
+    buildConf("kyuubi.backend.server.event.loggers")
       .doc("A comma separated list of server history loggers, where session/operation etc" +
         " events go.<ul>" +
         s" <li>JSON: the events will be written to the location of" +
@@ -1140,7 +1145,7 @@ object KyuubiConf {
       .createWithDefault(Nil)
 
   val ENGINE_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
-    buildConf("engine.event.loggers")
+    buildConf("kyuubi.engine.event.loggers")
       .doc("A comma separated list of engine history loggers, where engine/session/operation etc" +
         " events go. We use spark logger by default.<ul>" +
         " <li>SPARK: the events will be written to the spark listener bus.</li>" +
@@ -1158,14 +1163,14 @@ object KyuubiConf {
       .createWithDefault(Seq("SPARK"))
 
   val ENGINE_UI_STOP_ENABLED: ConfigEntry[Boolean] =
-    buildConf("engine.ui.stop.enabled")
+    buildConf("kyuubi.engine.ui.stop.enabled")
       .doc("When true, allows Kyuubi engine to be killed from the Spark Web UI.")
       .version("1.3.0")
       .booleanConf
       .createWithDefault(true)
 
   val ENGINE_UI_SESSION_LIMIT: ConfigEntry[Int] =
-    buildConf("engine.ui.retainedSessions")
+    buildConf("kyuubi.engine.ui.retainedSessions")
       .doc("The number of SQL client sessions kept in the Kyuubi Query Engine web UI.")
       .version("1.4.0")
       .intConf
@@ -1173,7 +1178,7 @@ object KyuubiConf {
       .createWithDefault(200)
 
   val ENGINE_UI_STATEMENT_LIMIT: ConfigEntry[Int] =
-    buildConf("engine.ui.retainedStatements")
+    buildConf("kyuubi.engine.ui.retainedStatements")
       .doc("The number of statements kept in the Kyuubi Query Engine web UI.")
       .version("1.4.0")
       .intConf
@@ -1181,28 +1186,28 @@ object KyuubiConf {
       .createWithDefault(200)
 
   val ENGINE_OPERATION_LOG_DIR_ROOT: ConfigEntry[String] =
-    buildConf("engine.operation.log.dir.root")
+    buildConf("kyuubi.engine.operation.log.dir.root")
       .doc("Root directory for query operation log at engine-side.")
       .version("1.4.0")
       .stringConf
       .createWithDefault("engine_operation_logs")
 
   val ENGINE_SECURITY_ENABLED: ConfigEntry[Boolean] =
-    buildConf("engine.security.enabled")
+    buildConf("kyuubi.engine.security.enabled")
       .doc("Whether to enable the internal secure access between Kyuubi server and engine.")
       .version("1.5.0")
       .booleanConf
       .createWithDefault(false)
 
   val ENGINE_SECURITY_TOKEN_MAX_LIFETIME: ConfigEntry[Long] =
-    buildConf("engine.security.token.max.lifetime")
+    buildConf("kyuubi.engine.security.token.max.lifetime")
       .doc("The max lifetime of the token used for secure access between Kyuubi server and engine.")
       .version("1.5.0")
       .timeConf
       .createWithDefault(Duration.ofMinutes(10).toMillis)
 
   val ENGINE_SECURITY_SECRET_PROVIDER: ConfigEntry[String] =
-    buildConf("engine.security.secret.provider")
+    buildConf("kyuubi.engine.security.secret.provider")
       .doc("The class used to manage the engine security secret. This class must be a " +
         "subclass of EngineSecuritySecretProvider.")
       .version("1.5.0")
@@ -1211,7 +1216,7 @@ object KyuubiConf {
         "org.apache.kyuubi.service.authentication.ZooKeeperEngineSecuritySecretProviderImpl")
 
   val ENGINE_SECURITY_CRYPTO_KEY_LENGTH: ConfigEntry[Int] =
-    buildConf("engine.security.crypto.keyLength")
+    buildConf("kyuubi.engine.security.crypto.keyLength")
       .doc("The length in bits of the encryption key to generate. " +
         "Valid values are 128, 192 and 256")
       .version("1.5.0")
@@ -1220,28 +1225,28 @@ object KyuubiConf {
       .createWithDefault(128)
 
   val ENGINE_SECURITY_CRYPTO_IV_LENGTH: ConfigEntry[Int] =
-    buildConf("engine.security.crypto.ivLength")
+    buildConf("kyuubi.engine.security.crypto.ivLength")
       .doc("Initial vector length, in bytes.")
       .version("1.5.0")
       .intConf
       .createWithDefault(16)
 
   val ENGINE_SECURITY_CRYPTO_KEY_ALGORITHM: ConfigEntry[String] =
-    buildConf("engine.security.crypto.keyAlgorithm")
+    buildConf("kyuubi.engine.security.crypto.keyAlgorithm")
       .doc("The algorithm for generated secret keys.")
       .version("1.5.0")
       .stringConf
       .createWithDefault("AES")
 
   val ENGINE_SECURITY_CRYPTO_CIPHER_TRANSFORMATION: ConfigEntry[String] =
-    buildConf("engine.security.crypto.cipher")
+    buildConf("kyuubi.engine.security.crypto.cipher")
       .doc("The cipher transformation to use for encrypting engine access token.")
       .version("1.5.0")
       .stringConf
       .createWithDefault("AES/CBC/PKCS5PADDING")
 
   val SESSION_NAME: OptionalConfigEntry[String] =
-    buildConf("session.name")
+    buildConf("kyuubi.session.name")
       .doc("A human readable name of session and we use empty string by default. " +
         "This name will be recorded in event. Note that, we only apply this value from " +
         "session conf.")
@@ -1255,7 +1260,7 @@ object KyuubiConf {
   }
 
   val OPERATION_PLAN_ONLY_MODE: ConfigEntry[String] =
-    buildConf("operation.plan.only.mode")
+    buildConf("kyuubi.operation.plan.only.mode")
       .doc("Whether to perform the statement in a PARSE, ANALYZE, OPTIMIZE, PHYSICAL, EXECUTION " +
         "only way without executing the query. When it is NONE, the statement will be fully " +
         "executed")
@@ -1266,7 +1271,7 @@ object KyuubiConf {
       .createWithDefault(OperationModes.NONE.toString)
 
   val OPERATION_PLAN_ONLY_EXCLUDES: ConfigEntry[Seq[String]] =
-    buildConf("operation.plan.only.excludes")
+    buildConf("kyuubi.operation.plan.only.excludes")
       .doc("Comma-separated list of query plan names, in the form of simple class names, i.e, " +
         "for `set abc=xyz`, the value will be `SetCommand`. For those auxiliary plans, such as " +
         "`switch databases`, `set properties`, or `create temporary view` e.t.c, " +
@@ -1284,7 +1289,7 @@ object KyuubiConf {
   }
 
   val OPERATION_LANGUAGE: ConfigEntry[String] =
-    buildConf("operation.language")
+    buildConf("kyuubi.operation.language")
       .doc("Choose a programing language for the following inputs" +
         " <ul><li>SQL: (Default) Run all following statements as SQL queries.</li>" +
         " <li>SCALA: Run all following input a scala codes</li></ul>")
@@ -1295,7 +1300,7 @@ object KyuubiConf {
       .createWithDefault(OperationLanguages.SQL.toString)
 
   val SESSION_CONF_ADVISOR: OptionalConfigEntry[String] =
-    buildConf("session.conf.advisor")
+    buildConf("kyuubi.session.conf.advisor")
       .doc("A config advisor plugin for Kyuubi Server. This plugin can provide some custom " +
         "configs for different user or session configs and overwrite the session configs before " +
         "open a new session. This config value should be a class which is a child of " +
@@ -1305,14 +1310,14 @@ object KyuubiConf {
       .createOptional
 
   val SERVER_NAME: OptionalConfigEntry[String] =
-    buildConf("server.name")
+    buildConf("kyuubi.server.name")
       .doc("The name of Kyuubi Server.")
       .version("1.5.0")
       .stringConf
       .createOptional
 
   val SERVER_CONF_IGNORE_PREFIX_LIST: ConfigEntry[Seq[String]] =
-    buildConf("server.conf.ignore.prefix.list")
+    buildConf("kyuubi.server.conf.ignore.prefix.list")
       .doc("A comma separated list of ignored keys prefix. If kyuubi conf contains any of" +
         " them, the key and the corresponding value will be removed silently during server" +
         " setup." +
@@ -1325,14 +1330,14 @@ object KyuubiConf {
       .createWithDefault(Seq("java", "sun", "os", "jdk"))
 
   val ENGINE_SPARK_SHOW_PROGRESS: ConfigEntry[Boolean] =
-    buildConf("session.engine.spark.showProgress")
+    buildConf("kyuubi.session.engine.spark.showProgress")
       .doc("When true, show the progress bar in the spark engine log.")
       .version("1.6.0")
       .booleanConf
       .createWithDefault(false)
 
   val ENGINE_SPARK_SHOW_PROGRESS_UPDATE_INTERVAL: ConfigEntry[Long] =
-    buildConf("session.engine.spark.progress.update.interval")
+    buildConf("kyuubi.session.engine.spark.progress.update.interval")
       .doc("Update period of progress bar.")
       .version("1.6.0")
       .timeConf
@@ -1340,7 +1345,7 @@ object KyuubiConf {
       .createWithDefault(1000)
 
   val ENGINE_SPARK_SHOW_PROGRESS_TIME_FORMAT: ConfigEntry[String] =
-    buildConf("session.engine.spark.progress.timeFormat")
+    buildConf("kyuubi.session.engine.spark.progress.timeFormat")
       .doc("The time format of the progress bar")
       .version("1.6.0")
       .stringConf
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/ConfigEntrySuite.scala b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/ConfigEntrySuite.scala
index f114d63..d1a7466 100644
--- a/kyuubi-common/src/test/scala/org/apache/kyuubi/config/ConfigEntrySuite.scala
+++ b/kyuubi-common/src/test/scala/org/apache/kyuubi/config/ConfigEntrySuite.scala
@@ -120,7 +120,7 @@ class ConfigEntrySuite extends KyuubiFunSuite {
   }
 
   test("fallback config entry") {
-    val origin = KyuubiConf.buildConf("origin.spark")
+    val origin = KyuubiConf.buildConf("kyuubi.origin.spark")
       .version("1.1.1")
       .stringConf.createWithDefault("origin")
     val fallback =
diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala
index 8ee3cc1..34a337c 100644
--- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala
+++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/HighAvailabilityConf.scala
@@ -28,13 +28,13 @@ object HighAvailabilityConf {
 
   private def buildConf(key: String): ConfigBuilder = KyuubiConf.buildConf(key)
 
-  val HA_ZK_QUORUM: ConfigEntry[String] = buildConf("ha.zookeeper.quorum")
+  val HA_ZK_QUORUM: ConfigEntry[String] = buildConf("kyuubi.ha.zookeeper.quorum")
     .doc("The connection string for the zookeeper ensemble")
     .version("1.0.0")
     .stringConf
     .createWithDefault("")
 
-  val HA_ZK_NAMESPACE: ConfigEntry[String] = buildConf("ha.zookeeper.namespace")
+  val HA_ZK_NAMESPACE: ConfigEntry[String] = buildConf("kyuubi.ha.zookeeper.namespace")
     .doc("The root directory for the service to deploy its instance uri")
     .version("1.0.0")
     .stringConf
@@ -42,14 +42,14 @@ object HighAvailabilityConf {
 
   @deprecated(s"using ${HA_ZK_AUTH_TYPE.key} and ${HA_ZK_ENGINE_AUTH_TYPE.key} instead", "1.3.2")
   val HA_ZK_ACL_ENABLED: ConfigEntry[Boolean] =
-    buildConf("ha.zookeeper.acl.enabled")
+    buildConf("kyuubi.ha.zookeeper.acl.enabled")
       .doc("Set to true if the zookeeper ensemble is kerberized")
       .version("1.0.0")
       .booleanConf
       .createWithDefault(UserGroupInformation.isSecurityEnabled)
 
   val HA_ZK_AUTH_TYPE: ConfigEntry[String] =
-    buildConf("ha.zookeeper.auth.type")
+    buildConf("kyuubi.ha.zookeeper.auth.type")
       .doc("The type of zookeeper authentication, all candidates are " +
         s"${ZooKeeperAuthTypes.values.mkString("<ul><li>", "</li><li> ", "</li></ul>")}")
       .version("1.3.2")
@@ -58,44 +58,45 @@ object HighAvailabilityConf {
       .createWithDefault(ZooKeeperAuthTypes.NONE.toString)
 
   val HA_ZK_ENGINE_AUTH_TYPE: ConfigEntry[String] =
-    buildConf("ha.zookeeper.engine.auth.type")
+    buildConf("kyuubi.ha.zookeeper.engine.auth.type")
       .doc("The type of zookeeper authentication for engine, all candidates are " +
         s"${ZooKeeperAuthTypes.values.mkString("<ul><li>", "</li><li> ", "</li></ul>")}")
       .version("1.3.2")
       .fallbackConf(HA_ZK_AUTH_TYPE)
 
-  val HA_ZK_AUTH_PRINCIPAL: ConfigEntry[Option[String]] = buildConf("ha.zookeeper.auth.principal")
-    .doc("Name of the Kerberos principal is used for zookeeper authentication.")
-    .version("1.3.2")
-    .fallbackConf(KyuubiConf.SERVER_PRINCIPAL)
+  val HA_ZK_AUTH_PRINCIPAL: ConfigEntry[Option[String]] =
+    buildConf("kyuubi.ha.zookeeper.auth.principal")
+      .doc("Name of the Kerberos principal is used for zookeeper authentication.")
+      .version("1.3.2")
+      .fallbackConf(KyuubiConf.SERVER_PRINCIPAL)
 
-  val HA_ZK_AUTH_KEYTAB: ConfigEntry[Option[String]] = buildConf("ha.zookeeper.auth.keytab")
+  val HA_ZK_AUTH_KEYTAB: ConfigEntry[Option[String]] = buildConf("kyuubi.ha.zookeeper.auth.keytab")
     .doc("Location of Kyuubi server's keytab is used for zookeeper authentication.")
     .version("1.3.2")
     .fallbackConf(KyuubiConf.SERVER_KEYTAB)
 
-  val HA_ZK_AUTH_DIGEST: OptionalConfigEntry[String] = buildConf("ha.zookeeper.auth.digest")
+  val HA_ZK_AUTH_DIGEST: OptionalConfigEntry[String] = buildConf("kyuubi.ha.zookeeper.auth.digest")
     .doc("The digest auth string is used for zookeeper authentication, like: username:password.")
     .version("1.3.2")
     .stringConf
     .createOptional
 
   val HA_ZK_CONN_MAX_RETRIES: ConfigEntry[Int] =
-    buildConf("ha.zookeeper.connection.max.retries")
+    buildConf("kyuubi.ha.zookeeper.connection.max.retries")
       .doc("Max retry times for connecting to the zookeeper ensemble")
       .version("1.0.0")
       .intConf
       .createWithDefault(3)
 
   val HA_ZK_CONN_BASE_RETRY_WAIT: ConfigEntry[Int] =
-    buildConf("ha.zookeeper.connection.base.retry.wait")
+    buildConf("kyuubi.ha.zookeeper.connection.base.retry.wait")
       .doc("Initial amount of time to wait between retries to the zookeeper ensemble")
       .version("1.0.0")
       .intConf
       .createWithDefault(1000)
 
   val HA_ZK_CONN_MAX_RETRY_WAIT: ConfigEntry[Int] =
-    buildConf("ha.zookeeper.connection.max.retry.wait")
+    buildConf("kyuubi.ha.zookeeper.connection.max.retry.wait")
       .doc(s"Max amount of time to wait between retries for" +
         s" ${RetryPolicies.BOUNDED_EXPONENTIAL_BACKOFF} policy can reach, or max time until" +
         s" elapsed for ${RetryPolicies.UNTIL_ELAPSED} policy to connect the zookeeper ensemble")
@@ -103,20 +104,20 @@ object HighAvailabilityConf {
       .intConf
       .createWithDefault(30 * 1000)
 
-  val HA_ZK_CONN_TIMEOUT: ConfigEntry[Int] = buildConf("ha.zookeeper.connection.timeout")
+  val HA_ZK_CONN_TIMEOUT: ConfigEntry[Int] = buildConf("kyuubi.ha.zookeeper.connection.timeout")
     .doc("The timeout(ms) of creating the connection to the zookeeper ensemble")
     .version("1.0.0")
     .intConf
     .createWithDefault(15 * 1000)
 
-  val HA_ZK_SESSION_TIMEOUT: ConfigEntry[Int] = buildConf("ha.zookeeper.session.timeout")
+  val HA_ZK_SESSION_TIMEOUT: ConfigEntry[Int] = buildConf("kyuubi.ha.zookeeper.session.timeout")
     .doc("The timeout(ms) of a connected session to be idled")
     .version("1.0.0")
     .intConf
     .createWithDefault(60 * 1000)
 
   val HA_ZK_CONN_RETRY_POLICY: ConfigEntry[String] =
-    buildConf("ha.zookeeper.connection.retry.policy")
+    buildConf("kyuubi.ha.zookeeper.connection.retry.policy")
       .doc("The retry policy for connecting to the zookeeper ensemble, all candidates are:" +
         s" ${RetryPolicies.values.mkString("<ul><li>", "</li><li> ", "</li></ul>")}")
       .version("1.0.0")
@@ -125,7 +126,7 @@ object HighAvailabilityConf {
       .createWithDefault(RetryPolicies.EXPONENTIAL_BACKOFF.toString)
 
   val HA_ZK_NODE_TIMEOUT: ConfigEntry[Long] =
-    buildConf("ha.zookeeper.node.creation.timeout")
+    buildConf("kyuubi.ha.zookeeper.node.creation.timeout")
       .doc("Timeout for creating zookeeper node")
       .version("1.2.0")
       .timeConf
@@ -133,7 +134,7 @@ object HighAvailabilityConf {
       .createWithDefault(Duration.ofSeconds(120).toMillis)
 
   val HA_ZK_ENGINE_REF_ID: OptionalConfigEntry[String] =
-    buildConf("ha.engine.ref.id")
+    buildConf("kyuubi.ha.engine.ref.id")
       .doc("The engine reference id will be attached to zookeeper node when engine started, " +
         "and the kyuubi server will check it cyclically.")
       .internal
@@ -142,7 +143,7 @@ object HighAvailabilityConf {
       .createOptional
 
   val HA_ZK_PUBLISH_CONFIGS: ConfigEntry[Boolean] =
-    buildConf("ha.zookeeper.publish.configs")
+    buildConf("kyuubi.ha.zookeeper.publish.configs")
       .doc("When set to true, publish Kerberos configs to Zookeeper." +
         "Note that the Hive driver needs to be greater than 1.3 or 2.0 or apply HIVE-11581 patch.")
       .version("1.4.0")
@@ -150,7 +151,7 @@ object HighAvailabilityConf {
       .createWithDefault(false)
 
   val HA_ZK_ENGINE_SECURE_SECRET_NODE: OptionalConfigEntry[String] =
-    buildConf("ha.zookeeper.engine.secure.secret.node")
+    buildConf("kyuubi.ha.zookeeper.engine.secure.secret.node")
       .doc("The zk node contains the secret that used for internal secure between Kyuubi server " +
         "and Kyuubi engine, please make sure that it is only visible for Kyuubi.")
       .version("1.5.0")
diff --git a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
index 8f75c31..cacc15b 100644
--- a/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
+++ b/kyuubi-metrics/src/main/scala/org/apache/kyuubi/metrics/MetricsConf.scala
@@ -27,13 +27,13 @@ object MetricsConf {
   private def buildConf(key: String): ConfigBuilder = KyuubiConf.buildConf(key)
 
   val METRICS_ENABLED: ConfigEntry[Boolean] =
-    buildConf("metrics.enabled")
+    buildConf("kyuubi.metrics.enabled")
       .doc("Set to true to enable kyuubi metrics system")
       .version("1.2.0")
       .booleanConf
       .createWithDefault(true)
 
-  val METRICS_REPORTERS: ConfigEntry[Seq[String]] = buildConf("metrics.reporters")
+  val METRICS_REPORTERS: ConfigEntry[Seq[String]] = buildConf("kyuubi.metrics.reporters")
     .doc("A comma separated list for all metrics reporters" +
       "<ul>" +
       " <li>CONSOLE - ConsoleReporter which outputs measurements to CONSOLE periodically.</li>" +
@@ -51,39 +51,39 @@ object MetricsConf {
       s"the reporter type should be one or more of ${ReporterType.values.mkString(",")}")
     .createWithDefault(Seq(JSON.toString))
 
-  val METRICS_CONSOLE_INTERVAL: ConfigEntry[Long] = buildConf("metrics.console.interval")
+  val METRICS_CONSOLE_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.metrics.console.interval")
     .doc("How often should report metrics to console")
     .version("1.2.0")
     .timeConf
     .createWithDefault(Duration.ofSeconds(5).toMillis)
 
-  val METRICS_JSON_LOCATION: ConfigEntry[String] = buildConf("metrics.json.location")
+  val METRICS_JSON_LOCATION: ConfigEntry[String] = buildConf("kyuubi.metrics.json.location")
     .doc("Where the json metrics file located")
     .version("1.2.0")
     .stringConf
     .createWithDefault("metrics")
 
-  val METRICS_JSON_INTERVAL: ConfigEntry[Long] = buildConf("metrics.json.interval")
+  val METRICS_JSON_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.metrics.json.interval")
     .doc("How often should report metrics to json file")
     .version("1.2.0")
     .timeConf
     .createWithDefault(Duration.ofSeconds(5).toMillis)
 
-  val METRICS_PROMETHEUS_PORT: ConfigEntry[Int] = buildConf("metrics.prometheus.port")
+  val METRICS_PROMETHEUS_PORT: ConfigEntry[Int] = buildConf("kyuubi.metrics.prometheus.port")
     .doc("Prometheus metrics HTTP server port")
     .version("1.2.0")
     .intConf
     .checkValue(p => p == 0 || (p > 1024 && p < 65535), "Invalid Port number")
     .createWithDefault(10019)
 
-  val METRICS_PROMETHEUS_PATH: ConfigEntry[String] = buildConf("metrics.prometheus.path")
+  val METRICS_PROMETHEUS_PATH: ConfigEntry[String] = buildConf("kyuubi.metrics.prometheus.path")
     .doc("URI context path of prometheus metrics HTTP server")
     .version("1.2.0")
     .stringConf
     .checkValue(path => path.startsWith("/"), "Context path must start with '/'")
     .createWithDefault("/metrics")
 
-  val METRICS_SLF4J_INTERVAL: ConfigEntry[Long] = buildConf("metrics.slf4j.interval")
+  val METRICS_SLF4J_INTERVAL: ConfigEntry[Long] = buildConf("kyuubi.metrics.slf4j.interval")
     .doc("How often should report metrics to SLF4J logger")
     .version("1.2.0")
     .timeConf
diff --git a/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/ZookeeperConf.scala b/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/ZookeeperConf.scala
index 07625f1..9acd5ab 100644
--- a/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/ZookeeperConf.scala
+++ b/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/ZookeeperConf.scala
@@ -24,51 +24,51 @@ object ZookeeperConf {
   private def buildConf(key: String): ConfigBuilder = KyuubiConf.buildConf(key)
 
   @deprecated(s"using ${ZK_CLIENT_PORT.key} instead", since = "1.2.0")
-  val EMBEDDED_ZK_PORT: ConfigEntry[Int] = buildConf("zookeeper.embedded.port")
+  val EMBEDDED_ZK_PORT: ConfigEntry[Int] = buildConf("kyuubi.zookeeper.embedded.port")
     .doc("The port of the embedded zookeeper server")
     .version("1.0.0")
     .intConf
     .createWithDefault(2181)
 
   @deprecated(s"using ${ZK_DATA_DIR.key} instead", since = "1.2.0")
-  val EMBEDDED_ZK_TEMP_DIR: ConfigEntry[String] = buildConf("zookeeper.embedded.directory")
+  val EMBEDDED_ZK_TEMP_DIR: ConfigEntry[String] = buildConf("kyuubi.zookeeper.embedded.directory")
     .doc("The temporary directory for the embedded zookeeper server")
     .version("1.0.0")
     .stringConf
     .createWithDefault("embedded_zookeeper")
 
-  val ZK_CLIENT_PORT: ConfigEntry[Int] = buildConf("zookeeper.embedded.client.port")
+  val ZK_CLIENT_PORT: ConfigEntry[Int] = buildConf("kyuubi.zookeeper.embedded.client.port")
     .doc("clientPort for the embedded zookeeper server to listen for client connections," +
       " a client here could be Kyuubi server, engine and JDBC client")
     .version("1.2.0")
     .fallbackConf(EMBEDDED_ZK_PORT)
 
   val ZK_CLIENT_PORT_ADDRESS: OptionalConfigEntry[String] =
-    buildConf("zookeeper.embedded.client.port.address")
+    buildConf("kyuubi.zookeeper.embedded.client.port.address")
       .doc("clientPortAddress for the embedded zookeeper server to")
       .version("1.2.0")
       .stringConf
       .createOptional
 
-  val ZK_DATA_DIR: ConfigEntry[String] = buildConf("zookeeper.embedded.data.dir")
+  val ZK_DATA_DIR: ConfigEntry[String] = buildConf("kyuubi.zookeeper.embedded.data.dir")
     .doc("dataDir for the embedded zookeeper server where stores the in-memory database" +
       " snapshots and, unless specified otherwise, the transaction log of updates to the database.")
     .version("1.2.0")
     .fallbackConf(EMBEDDED_ZK_TEMP_DIR)
 
-  val ZK_DATA_LOG_DIR: ConfigEntry[String] = buildConf("zookeeper.embedded.data.log.dir")
+  val ZK_DATA_LOG_DIR: ConfigEntry[String] = buildConf("kyuubi.zookeeper.embedded.data.log.dir")
     .doc("dataLogDir for the embedded zookeeper server where writes the transaction log .")
     .version("1.2.0")
     .fallbackConf(ZK_DATA_DIR)
 
-  val ZK_TICK_TIME: ConfigEntry[Int] = buildConf("zookeeper.embedded.tick.time")
+  val ZK_TICK_TIME: ConfigEntry[Int] = buildConf("kyuubi.zookeeper.embedded.tick.time")
     .doc("tickTime in milliseconds for the embedded zookeeper server")
     .version("1.2.0")
     .intConf
     .createWithDefault(3000)
 
   val ZK_MAX_CLIENT_CONNECTIONS: ConfigEntry[Int] =
-    buildConf("zookeeper.embedded.max.client.connections")
+    buildConf("kyuubi.zookeeper.embedded.max.client.connections")
       .doc("maxClientCnxns for the embedded zookeeper server to limits the number of concurrent" +
         " connections of a single client identified by IP address")
       .version("1.2.0")
@@ -76,7 +76,7 @@ object ZookeeperConf {
       .createWithDefault(120)
 
   val ZK_MIN_SESSION_TIMEOUT: ConfigEntry[Int] =
-    buildConf("zookeeper.embedded.min.session.timeout")
+    buildConf("kyuubi.zookeeper.embedded.min.session.timeout")
       .doc("minSessionTimeout in milliseconds for the embedded zookeeper server will allow the" +
         " client to negotiate. Defaults to 2 times the tickTime")
       .version("1.2.0")
@@ -84,7 +84,7 @@ object ZookeeperConf {
       .createWithDefault(3000 * 2)
 
   val ZK_MAX_SESSION_TIMEOUT: ConfigEntry[Int] =
-    buildConf("zookeeper.embedded.max.session.timeout")
+    buildConf("kyuubi.zookeeper.embedded.max.session.timeout")
       .doc("maxSessionTimeout in milliseconds for the embedded zookeeper server will allow the" +
         " client to negotiate. Defaults to 20 times the tickTime")
       .version("1.2.0")