You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by fe...@apache.org on 2022/07/13 12:30:47 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #3045] Support to do admin rest request with kyuubi-adminctl

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

feiwang 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 a7d190dd7 [KYUUBI #3045] Support to do admin rest request with kyuubi-adminctl
a7d190dd7 is described below

commit a7d190dd7b92a5e28f6247fe91fdf3e4d7244973
Author: Fei Wang <fw...@ebay.com>
AuthorDate: Wed Jul 13 20:30:39 2022 +0800

    [KYUUBI #3045] Support to do admin rest request with kyuubi-adminctl
    
    ### _Why are the changes needed?_
    
    Close #3045
    
    Introduce kyuubi-adminctl for admin control.
    
    ```
    Usage: kyuubi-adminctl [refresh] [options]
    
      -v, --version <value>    Using the compiled KYUUBI_VERSION default, change it if the active service is running in another.
      -b, --verbose            Print additional debug output.
      --hostUrl <value>        Host url for rest api.
      --authSchema <value>     Auth schema for rest api, valid values are basic, spnego.
      --username <value>       Username for basic authentication.
      --password <value>       Password for basic authentication.
      --spnegoHost <value>     Spnego host for spnego authentication.
      --conf <value>           Kyuubi config property pair, formatted key=value.
    
    Command: refresh [config] <args>...
            Refresh the resource.
    Command: refresh config [<configType>]
            Refresh the config with specified type.
      <configType>             The valid config type can be one of the following: hadoopConf.
    
      -h, --help               Show help message and exit.
    ```
    
    Now support `kyuubi-adminctl refresh config hadoopConf` to reload hadoop conf.
    
    ### _How was this patch tested?_
    - [x] 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 #3057 from turboFei/kyuubi_admin_ctl.
    
    Closes #3045
    
    b6b46f3a [Fei Wang] refactor the info render
    d3b600f2 [Fei Wang] add toString
    234ae84c [Fei Wang] fix ut
    7db818d0 [Fei Wang] save
    e9e50a0f [Fei Wang] add ut
    849edad8 [Fei Wang] add ut
    c4d9b784 [Fei Wang] fix ut
    1c8d7c9b [Fei Wang] add kyuubi-adminctl
    8b1921ae [Fei Wang] save
    155f57af [Fei Wang] saev
    2ca8b665 [Fei Wang] save
    5ad40cea [Fei Wang] save
    bb2e1178 [Fei Wang] add refresh config
    
    Authored-by: Fei Wang <fw...@ebay.com>
    Signed-off-by: Fei Wang <fw...@ebay.com>
---
 bin/kyuubi-adminctl                                | 48 +++++++++++
 .../org/apache/kyuubi/ctl/AdminControlCli.scala    | 63 ++++++++++++++
 .../kyuubi/ctl/AdminControlCliArguments.scala      | 54 ++++++++++++
 .../scala/org/apache/kyuubi/ctl/CliConfig.scala    | 18 ++--
 .../scala/org/apache/kyuubi/ctl/CommandLine.scala  | 72 +++++++++++++---
 .../apache/kyuubi/ctl/ControlCliArguments.scala    | 20 ++---
 .../apache/kyuubi/ctl/cmd/AdminCtlCommand.scala    | 27 ++++++
 .../scala/org/apache/kyuubi/ctl/cmd/Command.scala  | 27 +++---
 .../ctl/cmd/create/CreateServerCommand.scala       | 10 +--
 .../kyuubi/ctl/cmd/delete/DeleteCommand.scala      |  2 +-
 .../ctl/cmd/refresh/RefreshConfigCommand.scala     | 45 ++++++++++
 .../org/apache/kyuubi/ctl/util/CtlUtils.scala      |  6 +-
 .../org/apache/kyuubi/ctl/util/Validator.scala     | 22 +++--
 .../kyuubi/ctl/AdminControlCliArgumentsSuite.scala | 99 ++++++++++++++++++++++
 .../kyuubi/ctl/ControlCliArgumentsSuite.scala      | 36 ++++----
 .../org/apache/kyuubi/ctl/TestPrematureExit.scala  | 26 ++++++
 .../org/apache/kyuubi/client/AdminRestApi.java     | 39 +++++++++
 .../java/org/apache/kyuubi/client/RestClient.java  |  6 +-
 .../kyuubi/server/api/v1/AdminResource.scala       |  2 +-
 .../kyuubi/server/rest/client/AdminCtlSuite.scala  | 44 ++++++++++
 .../server/rest/client/AdminRestApiSuite.scala     | 34 ++++++++
 21 files changed, 617 insertions(+), 83 deletions(-)

diff --git a/bin/kyuubi-adminctl b/bin/kyuubi-adminctl
new file mode 100755
index 000000000..74b5f39a1
--- /dev/null
+++ b/bin/kyuubi-adminctl
@@ -0,0 +1,48 @@
+#!/usr/bin/env bash
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+## Kyuubi Admin Control Client Entrance
+CLASS="org.apache.kyuubi.ctl.AdminControlCli"
+
+export KYUUBI_HOME="$(cd "$(dirname "$0")"/..; pwd)"
+
+. "${KYUUBI_HOME}/bin/load-kyuubi-env.sh" -s
+
+if [[ -z ${JAVA_HOME} ]]; then
+  echo "Error: JAVA_HOME IS NOT SET! CANNOT PROCEED."
+  exit 1
+fi
+
+RUNNER="${JAVA_HOME}/bin/java"
+
+## Find the Kyuubi Jar
+if [[ -z "$KYUUBI_JAR_DIR" ]]; then
+  KYUUBI_JAR_DIR="$KYUUBI_HOME/jars"
+  if [[ ! -d ${KYUUBI_JAR_DIR} ]]; then
+  echo -e "\nCandidate Kyuubi lib $KYUUBI_JAR_DIR doesn't exist, searching development environment..."
+    KYUUBI_JAR_DIR="$KYUUBI_HOME/kyuubi-assembly/target/scala-${KYUUBI_SCALA_VERSION}/jars"
+  fi
+fi
+
+if [[ -z ${YARN_CONF_DIR} ]]; then
+  KYUUBI_CLASSPATH="${KYUUBI_JAR_DIR}/*:${KYUUBI_CONF_DIR}:${HADOOP_CONF_DIR}"
+else
+  KYUUBI_CLASSPATH="${KYUUBI_JAR_DIR}/*:${KYUUBI_CONF_DIR}:${HADOOP_CONF_DIR}:${YARN_CONF_DIR}"
+fi
+
+exec ${RUNNER} ${KYUUBI_CTL_JAVA_OPTS} -cp ${KYUUBI_CLASSPATH} $CLASS "$@"
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCli.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCli.scala
new file mode 100644
index 000000000..d857febb1
--- /dev/null
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCli.scala
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.ctl
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.ctl.util.CommandLineUtils
+
+class AdminControlCli extends ControlCli {
+  override protected def parseArguments(args: Array[String]): AdminControlCliArguments = {
+    new AdminControlCliArguments(args)
+  }
+}
+
+object AdminControlCli extends CommandLineUtils with Logging {
+  override def main(args: Array[String]): Unit = {
+    val adminCtl = new AdminControlCli() { self =>
+      override protected def parseArguments(args: Array[String]): AdminControlCliArguments = {
+        new AdminControlCliArguments(args) {
+          override def info(msg: => Any): Unit = self.info(msg)
+          override def warn(msg: => Any): Unit = self.warn(msg)
+          override def error(msg: => Any): Unit = self.error(msg)
+
+          override private[kyuubi] lazy val effectSetup = new KyuubiOEffectSetup {
+            override def displayToOut(msg: String): Unit = self.info(msg)
+            override def displayToErr(msg: String): Unit = self.error(msg)
+            override def reportWarning(msg: String): Unit = self.warn(msg)
+            override def reportError(msg: String): Unit = self.error(msg)
+          }
+        }
+      }
+
+      override def info(msg: => Any): Unit = printMessage(msg)
+      override def warn(msg: => Any): Unit = printMessage(s"Warning: $msg")
+      override def error(msg: => Any): Unit = printMessage(s"Error: $msg")
+
+      override def doAction(args: Array[String]): Unit = {
+        try {
+          super.doAction(args)
+          exitFn(0)
+        } catch {
+          case e: ControlCliException => exitFn(e.exitCode)
+        }
+      }
+    }
+
+    adminCtl.doAction(args)
+  }
+}
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCliArguments.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCliArguments.scala
new file mode 100644
index 000000000..9bc33808e
--- /dev/null
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/AdminControlCliArguments.scala
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.ctl
+
+import scopt.OParser
+
+import org.apache.kyuubi.KyuubiException
+import org.apache.kyuubi.ctl.cmd.Command
+import org.apache.kyuubi.ctl.cmd.refresh.RefreshConfigCommand
+
+class AdminControlCliArguments(args: Seq[String], env: Map[String, String] = sys.env)
+  extends ControlCliArguments(args, env) {
+  override def parser(): OParser[Unit, CliConfig] = {
+    val builder = OParser.builder[CliConfig]
+    CommandLine.getAdminCtlOptionParser(builder)
+  }
+
+  override protected def getCommand(cliConfig: CliConfig): Command[_] = {
+    cliConfig.action match {
+      case ControlAction.REFRESH => cliConfig.resource match {
+          case ControlObject.CONFIG => new RefreshConfigCommand(cliConfig)
+          case _ => throw new KyuubiException(s"Invalid resource: ${cliConfig.resource}")
+        }
+      case _ => throw new KyuubiException(s"Invalid operation: ${cliConfig.action}")
+    }
+  }
+
+  override def toString: String = {
+    cliConfig.resource match {
+      case ControlObject.CONFIG =>
+        s"""Parsed arguments:
+           |  action                  ${cliConfig.action}
+           |  resource                ${cliConfig.resource}
+           |  configType              ${cliConfig.adminConfigOpts.configType}
+        """.stripMargin
+      case _ => ""
+    }
+  }
+}
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CliConfig.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CliConfig.scala
index d5c8d49e5..873a4b1a1 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CliConfig.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CliConfig.scala
@@ -21,29 +21,27 @@ import org.apache.kyuubi.ctl.ControlObject.ControlObject
 
 private[ctl] object ControlAction extends Enumeration {
   type ControlAction = Value
-  val CREATE, GET, DELETE, LIST, LOG, SUBMIT = Value
+  val CREATE, GET, DELETE, LIST, LOG, SUBMIT, REFRESH = Value
 }
 
 private[ctl] object ControlObject extends Enumeration {
   type ControlObject = Value
-  val SERVER, ENGINE, BATCH = Value
+  val SERVER, ENGINE, BATCH, CONFIG = Value
 }
 
 case class CliConfig(
     action: ControlAction = null,
     resource: ControlObject = ControlObject.SERVER,
     commonOpts: CommonOpts = CommonOpts(),
+    zkOpts: ZookeeperOpts = ZookeeperOpts(),
     createOpts: CreateOpts = CreateOpts(),
     logOpts: LogOpts = LogOpts(),
     batchOpts: BatchOpts = BatchOpts(),
     engineOpts: EngineOpts = EngineOpts(),
+    adminConfigOpts: AdminConfigOpts = AdminConfigOpts(),
     conf: Map[String, String] = Map.empty)
 
 case class CommonOpts(
-    zkQuorum: String = null,
-    namespace: String = null,
-    host: String = null,
-    port: String = null,
     version: String = null,
     verbose: Boolean = false,
     hostUrl: String = null,
@@ -52,6 +50,12 @@ case class CommonOpts(
     password: String = null,
     spnegoHost: String = null)
 
+case class ZookeeperOpts(
+    zkQuorum: String = null,
+    namespace: String = null,
+    host: String = null,
+    port: String = null)
+
 case class CreateOpts(filename: String = null)
 
 case class LogOpts(forward: Boolean = false)
@@ -73,3 +77,5 @@ case class EngineOpts(
     engineType: String = null,
     engineSubdomain: String = null,
     engineShareLevel: String = null)
+
+case class AdminConfigOpts(configType: String = null)
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CommandLine.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CommandLine.scala
index 2dc8be824..351e0d28a 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CommandLine.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/CommandLine.scala
@@ -23,12 +23,13 @@ import org.apache.kyuubi.ctl.util.DateTimeUtils._
 
 object CommandLine {
 
-  def getOptionParser(builder: OParserBuilder[CliConfig]): OParser[Unit, CliConfig] = {
+  def getCtlOptionParser(builder: OParserBuilder[CliConfig]): OParser[Unit, CliConfig] = {
     import builder._
     OParser.sequence(
       programName("kyuubi-ctl"),
       head("kyuubi", KYUUBI_VERSION),
       common(builder),
+      zooKeeper(builder),
       create(builder),
       get(builder),
       delete(builder),
@@ -46,22 +47,27 @@ object CommandLine {
       help('h', "help").text("Show help message and exit."))
   }
 
+  def getAdminCtlOptionParser(builder: OParserBuilder[CliConfig]): OParser[Unit, CliConfig] = {
+    import builder._
+    OParser.sequence(
+      programName("kyuubi-adminctl"),
+      head("kyuubi", KYUUBI_VERSION),
+      common(builder),
+      refresh(builder),
+      checkConfig(f => {
+        if (f.action == null) {
+          failure("Must specify action command: [refresh].")
+        } else {
+          success
+        }
+      }),
+      note(""),
+      help('h', "help").text("Show help message and exit."))
+  }
+
   private def common(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
     import builder._
     OParser.sequence(
-      opt[String]("zk-quorum").abbr("zk")
-        .action((v, c) => c.copy(commonOpts = c.commonOpts.copy(zkQuorum = v)))
-        .text("The connection string for the zookeeper ensemble," +
-          " using zk quorum manually."),
-      opt[String]('n', "namespace")
-        .action((v, c) => c.copy(commonOpts = c.commonOpts.copy(namespace = v)))
-        .text("The namespace, using kyuubi-defaults/conf if absent."),
-      opt[String]('s', "host")
-        .action((v, c) => c.copy(commonOpts = c.commonOpts.copy(host = v)))
-        .text("Hostname or IP address of a service."),
-      opt[String]('p', "port")
-        .action((v, c) => c.copy(commonOpts = c.commonOpts.copy(port = v)))
-        .text("Listening port of a service."),
       opt[String]('v', "version")
         .action((v, c) => c.copy(commonOpts = c.commonOpts.copy(version = v)))
         .text("Using the compiled KYUUBI_VERSION default," +
@@ -94,6 +100,24 @@ object CommandLine {
         .text("Kyuubi config property pair, formatted key=value."))
   }
 
+  private def zooKeeper(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
+    import builder._
+    OParser.sequence(
+      opt[String]("zk-quorum").abbr("zk")
+        .action((v, c) => c.copy(zkOpts = c.zkOpts.copy(zkQuorum = v)))
+        .text("The connection string for the zookeeper ensemble," +
+          " using zk quorum manually."),
+      opt[String]('n', "namespace")
+        .action((v, c) => c.copy(zkOpts = c.zkOpts.copy(namespace = v)))
+        .text("The namespace, using kyuubi-defaults/conf if absent."),
+      opt[String]('s', "host")
+        .action((v, c) => c.copy(zkOpts = c.zkOpts.copy(host = v)))
+        .text("Hostname or IP address of a service."),
+      opt[String]('p', "port")
+        .action((v, c) => c.copy(zkOpts = c.zkOpts.copy(port = v)))
+        .text("Listening port of a service."))
+  }
+
   private def create(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
     import builder._
     OParser.sequence(
@@ -179,6 +203,17 @@ object CommandLine {
           submitBatchCmd(builder).text("\topen batch session and wait for completion.")))
   }
 
+  private def refresh(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
+    import builder._
+    OParser.sequence(
+      note(""),
+      cmd("refresh")
+        .text("\tRefresh the resource.")
+        .action((_, c) => c.copy(action = ControlAction.REFRESH))
+        .children(
+          refreshConfigCmd(builder).text("\tRefresh the config with specified type.")))
+  }
+
   private def serverCmd(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
     import builder._
     cmd("server").action((_, c) => c.copy(resource = ControlObject.SERVER))
@@ -318,4 +353,13 @@ object CommandLine {
             "when the batch is no longer in PENDING state."))
   }
 
+  private def refreshConfigCmd(builder: OParserBuilder[CliConfig]): OParser[_, CliConfig] = {
+    import builder._
+    cmd("config").action((_, c) => c.copy(resource = ControlObject.CONFIG))
+      .children(
+        arg[String]("<configType>")
+          .optional()
+          .action((v, c) => c.copy(adminConfigOpts = c.adminConfigOpts.copy(configType = v)))
+          .text("The valid config type can be one of the following: hadoopConf."))
+  }
 }
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/ControlCliArguments.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/ControlCliArguments.scala
index 332c03a75..6254ba900 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/ControlCliArguments.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/ControlCliArguments.scala
@@ -42,7 +42,7 @@ class ControlCliArguments(args: Seq[String], env: Map[String, String] = sys.env)
 
   override def parser(): OParser[Unit, CliConfig] = {
     val builder = OParser.builder[CliConfig]
-    CommandLine.getOptionParser(builder)
+    CommandLine.getCtlOptionParser(builder)
   }
 
   private[kyuubi] lazy val effectSetup = new KyuubiOEffectSetup
@@ -62,7 +62,7 @@ class ControlCliArguments(args: Seq[String], env: Map[String, String] = sys.env)
     }
   }
 
-  private def getCommand(cliConfig: CliConfig): Command[_] = {
+  protected def getCommand(cliConfig: CliConfig): Command[_] = {
     cliConfig.action match {
       case ControlAction.CREATE => cliConfig.resource match {
           case ControlObject.BATCH => new CreateBatchCommand(cliConfig)
@@ -118,10 +118,10 @@ class ControlCliArguments(args: Seq[String], env: Map[String, String] = sys.env)
         s"""Parsed arguments:
            |  action                  ${cliConfig.action}
            |  resource                ${cliConfig.resource}
-           |  zkQuorum                ${cliConfig.commonOpts.zkQuorum}
-           |  namespace               ${cliConfig.commonOpts.namespace}
-           |  host                    ${cliConfig.commonOpts.host}
-           |  port                    ${cliConfig.commonOpts.port}
+           |  zkQuorum                ${cliConfig.zkOpts.zkQuorum}
+           |  namespace               ${cliConfig.zkOpts.namespace}
+           |  host                    ${cliConfig.zkOpts.host}
+           |  port                    ${cliConfig.zkOpts.port}
            |  version                 ${cliConfig.commonOpts.version}
            |  verbose                 ${cliConfig.commonOpts.verbose}
         """.stripMargin
@@ -129,11 +129,11 @@ class ControlCliArguments(args: Seq[String], env: Map[String, String] = sys.env)
         s"""Parsed arguments:
            |  action                  ${cliConfig.action}
            |  resource                ${cliConfig.resource}
-           |  zkQuorum                ${cliConfig.commonOpts.zkQuorum}
-           |  namespace               ${cliConfig.commonOpts.namespace}
+           |  zkQuorum                ${cliConfig.zkOpts.zkQuorum}
+           |  namespace               ${cliConfig.zkOpts.namespace}
            |  user                    ${cliConfig.engineOpts.user}
-           |  host                    ${cliConfig.commonOpts.host}
-           |  port                    ${cliConfig.commonOpts.port}
+           |  host                    ${cliConfig.zkOpts.host}
+           |  port                    ${cliConfig.zkOpts.port}
            |  version                 ${cliConfig.commonOpts.version}
            |  verbose                 ${cliConfig.commonOpts.verbose}
         """.stripMargin
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/AdminCtlCommand.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/AdminCtlCommand.scala
new file mode 100644
index 000000000..7e5eb5cd3
--- /dev/null
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/AdminCtlCommand.scala
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.ctl.cmd
+
+import org.apache.kyuubi.ctl.AdminControlCli
+import org.apache.kyuubi.ctl.CliConfig
+
+abstract class AdminCtlCommand[T](cliConfig: CliConfig) extends Command[T](cliConfig) {
+  override def info(msg: => Any): Unit = AdminControlCli.printMessage(msg)
+  override def warn(msg: => Any): Unit = AdminControlCli.printMessage(s"Warning: $msg")
+  override def error(msg: => Any): Unit = AdminControlCli.printMessage(s"Error: $msg")
+}
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/Command.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/Command.scala
index 7d0fc19a7..f0cdf71e0 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/Command.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/Command.scala
@@ -19,7 +19,7 @@ package org.apache.kyuubi.ctl.cmd
 import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiException, Logging}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.ctl.CliConfig
-import org.apache.kyuubi.ctl.ControlCli.printMessage
+import org.apache.kyuubi.ctl.ControlCli
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 
 abstract class Command[T](cliConfig: CliConfig) extends Logging {
@@ -50,27 +50,27 @@ abstract class Command[T](cliConfig: CliConfig) extends Logging {
   def fail(msg: String): Unit = throw new KyuubiException(msg)
 
   protected def mergeArgsIntoKyuubiConf(): Unit = {
-    conf.set(HA_ADDRESSES.key, normalizedCliConfig.commonOpts.zkQuorum)
-    conf.set(HA_NAMESPACE.key, normalizedCliConfig.commonOpts.namespace)
+    conf.set(HA_ADDRESSES.key, normalizedCliConfig.zkOpts.zkQuorum)
+    conf.set(HA_NAMESPACE.key, normalizedCliConfig.zkOpts.namespace)
   }
 
   private def useDefaultPropertyValueIfMissing(): CliConfig = {
     var arguments: CliConfig = cliConfig.copy()
-    if (cliConfig.commonOpts.zkQuorum == null) {
+    if (cliConfig.zkOpts.zkQuorum == null) {
       conf.getOption(HA_ADDRESSES.key).foreach { v =>
         if (verbose) {
           super.info(s"Zookeeper quorum is not specified, use value from default conf:$v")
         }
-        arguments = arguments.copy(commonOpts = arguments.commonOpts.copy(zkQuorum = v))
+        arguments = arguments.copy(zkOpts = arguments.zkOpts.copy(zkQuorum = v))
       }
     }
 
-    if (arguments.commonOpts.namespace == null) {
-      arguments = arguments.copy(commonOpts =
-        arguments.commonOpts.copy(namespace = conf.get(HA_NAMESPACE)))
+    if (arguments.zkOpts.namespace == null) {
+      arguments = arguments.copy(zkOpts =
+        arguments.zkOpts.copy(namespace = conf.get(HA_NAMESPACE)))
       if (verbose) {
         super.info(s"Zookeeper namespace is not specified, use value from default conf:" +
-          s"${arguments.commonOpts.namespace}")
+          s"${arguments.zkOpts.namespace}")
       }
     }
 
@@ -83,10 +83,7 @@ abstract class Command[T](cliConfig: CliConfig) extends Logging {
     arguments
   }
 
-  override def info(msg: => Any): Unit = printMessage(msg)
-
-  override def warn(msg: => Any): Unit = printMessage(s"Warning: $msg")
-
-  override def error(msg: => Any): Unit = printMessage(s"Error: $msg")
-
+  override def info(msg: => Any): Unit = ControlCli.printMessage(msg)
+  override def warn(msg: => Any): Unit = ControlCli.printMessage(s"Warning: $msg")
+  override def error(msg: => Any): Unit = ControlCli.printMessage(s"Error: $msg")
 }
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/create/CreateServerCommand.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/create/CreateServerCommand.scala
index 7f372f79e..3a942e0ab 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/create/CreateServerCommand.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/create/CreateServerCommand.scala
@@ -36,7 +36,7 @@ class CreateServerCommand(cliConfig: CliConfig) extends Command[Seq[ServiceNodeI
 
     val defaultNamespace = conf.getOption(HA_NAMESPACE.key)
       .getOrElse(HA_NAMESPACE.defaultValStr)
-    if (defaultNamespace.equals(normalizedCliConfig.commonOpts.namespace)) {
+    if (defaultNamespace.equals(normalizedCliConfig.zkOpts.namespace)) {
       fail(
         s"""
            |Only support expose Kyuubi server instance to another domain, a different namespace
@@ -52,7 +52,7 @@ class CreateServerCommand(cliConfig: CliConfig) extends Command[Seq[ServiceNodeI
   def doRun(): Seq[ServiceNodeInfo] = {
     val kyuubiConf = conf
 
-    kyuubiConf.setIfMissing(HA_ADDRESSES, normalizedCliConfig.commonOpts.zkQuorum)
+    kyuubiConf.setIfMissing(HA_ADDRESSES, normalizedCliConfig.zkOpts.zkQuorum)
     withDiscoveryClient(kyuubiConf) { discoveryClient =>
       val fromNamespace =
         DiscoveryPaths.makePath(null, kyuubiConf.get(HA_NAMESPACE))
@@ -68,7 +68,7 @@ class CreateServerCommand(cliConfig: CliConfig) extends Command[Seq[ServiceNodeI
               s" from $fromNamespace to $toNamespace")
             val newNodePath = zc.createAndGetServiceNode(
               kyuubiConf,
-              normalizedCliConfig.commonOpts.namespace,
+              normalizedCliConfig.zkOpts.namespace,
               sn.instance,
               sn.version,
               true)
@@ -78,10 +78,10 @@ class CreateServerCommand(cliConfig: CliConfig) extends Command[Seq[ServiceNodeI
           }
         }
 
-        if (kyuubiConf.get(HA_ADDRESSES) == normalizedCliConfig.commonOpts.zkQuorum) {
+        if (kyuubiConf.get(HA_ADDRESSES) == normalizedCliConfig.zkOpts.zkQuorum) {
           doCreate(discoveryClient)
         } else {
-          kyuubiConf.set(HA_ADDRESSES, normalizedCliConfig.commonOpts.zkQuorum)
+          kyuubiConf.set(HA_ADDRESSES, normalizedCliConfig.zkOpts.zkQuorum)
           withDiscoveryClient(kyuubiConf)(doCreate)
         }
       }
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/delete/DeleteCommand.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/delete/DeleteCommand.scala
index 309fefc8c..d42e23507 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/delete/DeleteCommand.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/delete/DeleteCommand.scala
@@ -39,7 +39,7 @@ class DeleteCommand(cliConfig: CliConfig) extends Command[Seq[ServiceNodeInfo]](
     withDiscoveryClient(conf) { discoveryClient =>
       val znodeRoot = CtlUtils.getZkNamespace(conf, normalizedCliConfig)
       val hostPortOpt =
-        Some((normalizedCliConfig.commonOpts.host, normalizedCliConfig.commonOpts.port.toInt))
+        Some((normalizedCliConfig.zkOpts.host, normalizedCliConfig.zkOpts.port.toInt))
       val nodesToDelete = CtlUtils.getServiceNodes(discoveryClient, znodeRoot, hostPortOpt)
 
       val deletedNodes = ListBuffer[ServiceNodeInfo]()
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala
new file mode 100644
index 000000000..18830f7b8
--- /dev/null
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/cmd/refresh/RefreshConfigCommand.scala
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.ctl.cmd.refresh
+
+import org.apache.kyuubi.KyuubiException
+import org.apache.kyuubi.client.AdminRestApi
+import org.apache.kyuubi.ctl.CliConfig
+import org.apache.kyuubi.ctl.RestClientFactory.withKyuubiRestClient
+import org.apache.kyuubi.ctl.cmd.AdminCtlCommand
+import org.apache.kyuubi.ctl.util.{Tabulator, Validator}
+
+class RefreshConfigCommand(cliConfig: CliConfig) extends AdminCtlCommand[String](cliConfig) {
+  def validate(): Unit = {
+    Validator.validateAdminConfigType(cliConfig)
+  }
+
+  def doRun(): String = {
+    withKyuubiRestClient(normalizedCliConfig, null, conf) { kyuubiRestClient =>
+      val adminRestApi = new AdminRestApi(kyuubiRestClient)
+      normalizedCliConfig.adminConfigOpts.configType match {
+        case "hadoopConf" => adminRestApi.refreshHadoopConf()
+        case configType => throw new KyuubiException(s"Invalid config type:$configType")
+      }
+    }
+  }
+
+  def render(resp: String): Unit = {
+    info(Tabulator.format("", Array("Response"), Array(Array(resp))))
+  }
+}
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/CtlUtils.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/CtlUtils.scala
index c957111d0..dde3c4038 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/CtlUtils.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/CtlUtils.scala
@@ -34,7 +34,7 @@ object CtlUtils {
   private[ctl] def getZkNamespace(conf: KyuubiConf, cliConfig: CliConfig): String = {
     cliConfig.resource match {
       case ControlObject.SERVER =>
-        DiscoveryPaths.makePath(null, cliConfig.commonOpts.namespace)
+        DiscoveryPaths.makePath(null, cliConfig.zkOpts.namespace)
       case ControlObject.ENGINE =>
         val engineType = Some(cliConfig.engineOpts.engineType)
           .filter(_ != null).filter(_.nonEmpty)
@@ -48,7 +48,7 @@ object CtlUtils {
         // The path of the engine defined in zookeeper comes from
         // org.apache.kyuubi.engine.EngineRef#engineSpace
         DiscoveryPaths.makePath(
-          s"${cliConfig.commonOpts.namespace}_" +
+          s"${cliConfig.zkOpts.namespace}_" +
             s"${cliConfig.commonOpts.version}_" +
             s"${engineShareLevel}_${engineType}",
           cliConfig.engineOpts.user,
@@ -81,7 +81,7 @@ object CtlUtils {
       val znodeRoot = getZkNamespace(conf, cliConfig)
       val hostPortOpt =
         if (filterHostPort) {
-          Some((cliConfig.commonOpts.host, cliConfig.commonOpts.port.toInt))
+          Some((cliConfig.zkOpts.host, cliConfig.zkOpts.port.toInt))
         } else None
       nodes = getServiceNodes(discoveryClient, znodeRoot, hostPortOpt)
     }
diff --git a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/Validator.scala b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/Validator.scala
index 738cdc2e3..05df08c1e 100644
--- a/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/Validator.scala
+++ b/kyuubi-ctl/src/main/scala/org/apache/kyuubi/ctl/util/Validator.scala
@@ -27,37 +27,37 @@ import org.apache.kyuubi.ctl.CliConfig
 private[ctl] object Validator {
 
   def validateZkArguments(cliConfig: CliConfig): Unit = {
-    if (cliConfig.commonOpts.zkQuorum == null) {
+    if (cliConfig.zkOpts.zkQuorum == null) {
       fail("Zookeeper quorum is not specified and no default value to load")
     }
-    if (cliConfig.commonOpts.namespace == null) {
+    if (cliConfig.zkOpts.namespace == null) {
       fail("Zookeeper namespace is not specified and no default value to load")
     }
   }
 
   def validateHostAndPort(cliConfig: CliConfig): Unit = {
-    if (cliConfig.commonOpts.host == null) {
+    if (cliConfig.zkOpts.host == null) {
       fail("Must specify host for service")
     }
-    if (cliConfig.commonOpts.port == null) {
+    if (cliConfig.zkOpts.port == null) {
       fail("Must specify port for service")
     }
 
     try {
-      InetAddress.getByName(cliConfig.commonOpts.host)
+      InetAddress.getByName(cliConfig.zkOpts.host)
     } catch {
       case _: Exception =>
-        fail(s"Unknown host: ${cliConfig.commonOpts.host}")
+        fail(s"Unknown host: ${cliConfig.zkOpts.host}")
     }
 
     try {
-      if (cliConfig.commonOpts.port.toInt <= 0) {
+      if (cliConfig.zkOpts.port.toInt <= 0) {
         fail(s"Specified port should be a positive number")
       }
     } catch {
       case _: NumberFormatException =>
         fail(s"Specified port is not a valid integer number: " +
-          s"${cliConfig.commonOpts.port}")
+          s"${cliConfig.zkOpts.port}")
     }
   }
 
@@ -72,5 +72,11 @@ private[ctl] object Validator {
     }
   }
 
+  def validateAdminConfigType(cliConfig: CliConfig): Unit = {
+    if (cliConfig.adminConfigOpts.configType == null) {
+      fail("The config type is not specified.")
+    }
+  }
+
   private def fail(msg: String): Unit = throw new KyuubiException(msg)
 }
diff --git a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala
new file mode 100644
index 000000000..4cf68d226
--- /dev/null
+++ b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/AdminControlCliArgumentsSuite.scala
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.ctl
+
+import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiFunSuite}
+
+class AdminControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
+
+  /** Check whether the script exits and the given search string is printed. */
+  private def testHelpExit(args: Array[String], searchString: String): Unit = {
+    val logAppender = new LogAppender("test premature exit")
+    withLogAppender(logAppender) {
+      val thread = new Thread {
+        override def run(): Unit =
+          try {
+            new AdminControlCliArguments(args) {
+              override private[kyuubi] lazy val effectSetup = new KyuubiOEffectSetup {
+                // nothing to do, to handle out stream.
+                override def terminate(exitState: Either[String, Unit]): Unit = ()
+              }
+            }
+          } catch {
+            case e: Exception =>
+              error(e)
+          }
+      }
+      thread.start()
+      thread.join()
+      assert(logAppender.loggingEvents.exists(
+        _.getMessage.getFormattedMessage.contains(searchString)))
+    }
+  }
+
+  test("test refresh config") {
+    var args = Array(
+      "refresh",
+      "server",
+      "hadoopConf")
+    testPrematureExitForAdminControlCliArgs(args, "Unknown argument 'server'")
+
+    args = Array(
+      "refresh",
+      "config",
+      "hadoopConf")
+    val opArgs = new AdminControlCliArguments(args)
+    assert(opArgs.cliConfig.action === ControlAction.REFRESH)
+    assert(opArgs.cliConfig.resource === ControlObject.CONFIG)
+    assert(opArgs.cliConfig.adminConfigOpts.configType === "hadoopConf")
+
+    args = Array(
+      "refresh",
+      "config",
+      "--hostUrl",
+      "https://kyuubi.test.com",
+      "otherConf")
+    testPrematureExitForAdminControlCli(args, "Invalid config type:otherConf")
+  }
+
+  test("test --help") {
+    // scalastyle:off
+    val helpString =
+      s"""kyuubi $KYUUBI_VERSION
+         |Usage: kyuubi-adminctl [refresh] [options]
+         |
+         |  -v, --version <value>    Using the compiled KYUUBI_VERSION default, change it if the active service is running in another.
+         |  -b, --verbose            Print additional debug output.
+         |  --hostUrl <value>        Host url for rest api.
+         |  --authSchema <value>     Auth schema for rest api, valid values are basic, spnego.
+         |  --username <value>       Username for basic authentication.
+         |  --password <value>       Password for basic authentication.
+         |  --spnegoHost <value>     Spnego host for spnego authentication.
+         |  --conf <value>           Kyuubi config property pair, formatted key=value.
+         |
+         |Command: refresh [config] <args>...
+         |	Refresh the resource.
+         |Command: refresh config [<configType>]
+         |	Refresh the config with specified type.
+         |  <configType>             The valid config type can be one of the following: hadoopConf.
+         |
+         |  -h, --help               Show help message and exit.""".stripMargin
+    // scalastyle:on
+    testHelpExit(Array("--help"), helpString)
+  }
+}
diff --git a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/ControlCliArgumentsSuite.scala b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/ControlCliArgumentsSuite.scala
index 1194e14b1..6241584e9 100644
--- a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/ControlCliArgumentsSuite.scala
+++ b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/ControlCliArgumentsSuite.scala
@@ -78,10 +78,10 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
         val opArgs = new ControlCliArguments(args)
         assert(opArgs.cliConfig.action.toString.equalsIgnoreCase(op))
         assert(opArgs.cliConfig.resource.toString.equalsIgnoreCase(service))
-        assert(opArgs.cliConfig.commonOpts.zkQuorum == zkQuorum)
-        assert(opArgs.cliConfig.commonOpts.namespace == namespace)
-        assert(opArgs.cliConfig.commonOpts.host == host)
-        assert(opArgs.cliConfig.commonOpts.port == port)
+        assert(opArgs.cliConfig.zkOpts.zkQuorum == zkQuorum)
+        assert(opArgs.cliConfig.zkOpts.namespace == namespace)
+        assert(opArgs.cliConfig.zkOpts.host == host)
+        assert(opArgs.cliConfig.zkOpts.port == port)
         assert(opArgs.cliConfig.commonOpts.version == KYUUBI_VERSION)
         if (service == "engine") {
           assert(opArgs.cliConfig.engineOpts.user == user)
@@ -109,10 +109,10 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
       val opArgs = new ControlCliArguments(args)
       assert(opArgs.cliConfig.action.toString.equalsIgnoreCase(op))
       assert(opArgs.cliConfig.resource.toString.equalsIgnoreCase(service))
-      assert(opArgs.cliConfig.commonOpts.zkQuorum == zkQuorum)
-      assert(opArgs.cliConfig.commonOpts.namespace == newNamespace)
-      assert(opArgs.cliConfig.commonOpts.host == host)
-      assert(opArgs.cliConfig.commonOpts.port == port)
+      assert(opArgs.cliConfig.zkOpts.zkQuorum == zkQuorum)
+      assert(opArgs.cliConfig.zkOpts.namespace == newNamespace)
+      assert(opArgs.cliConfig.zkOpts.host == host)
+      assert(opArgs.cliConfig.zkOpts.port == port)
       assert(opArgs.cliConfig.commonOpts.version == KYUUBI_VERSION)
     }
   }
@@ -295,7 +295,7 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
       "--zk-quorum",
       zkQuorum)
     val opArgs = new ControlCliArguments(args)
-    assert(opArgs.cliConfig.commonOpts.namespace == namespace)
+    assert(opArgs.cliConfig.zkOpts.namespace == namespace)
     assert(opArgs.cliConfig.commonOpts.version == KYUUBI_VERSION)
   }
 
@@ -324,10 +324,10 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
         val opArgs = new ControlCliArguments(args)
         assert(opArgs.cliConfig.action.toString.equalsIgnoreCase(op))
         assert(opArgs.cliConfig.resource.toString.equalsIgnoreCase(service))
-        assert(opArgs.cliConfig.commonOpts.zkQuorum == zkQuorum)
-        assert(opArgs.cliConfig.commonOpts.namespace == namespace)
-        assert(opArgs.cliConfig.commonOpts.host == host)
-        assert(opArgs.cliConfig.commonOpts.port == port)
+        assert(opArgs.cliConfig.zkOpts.zkQuorum == zkQuorum)
+        assert(opArgs.cliConfig.zkOpts.namespace == namespace)
+        assert(opArgs.cliConfig.zkOpts.host == host)
+        assert(opArgs.cliConfig.zkOpts.port == port)
         assert(opArgs.cliConfig.commonOpts.version == KYUUBI_VERSION)
         if (service == "engine") {
           assert(opArgs.cliConfig.engineOpts.user == user)
@@ -357,11 +357,6 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
       s"""kyuubi $KYUUBI_VERSION
          |Usage: kyuubi-ctl [create|get|delete|list|log|submit] [options]
          |
-         |  -zk, --zk-quorum <value>
-         |                           $zkHelpString
-         |  -n, --namespace <value>  The namespace, using kyuubi-defaults/conf if absent.
-         |  -s, --host <value>       Hostname or IP address of a service.
-         |  -p, --port <value>       Listening port of a service.
          |  -v, --version <value>    $versionHelpString
          |  -b, --verbose            Print additional debug output.
          |  --hostUrl <value>        Host url for rest api.
@@ -370,6 +365,11 @@ class ControlCliArgumentsSuite extends KyuubiFunSuite with TestPrematureExit {
          |  --password <value>       Password for basic authentication.
          |  --spnegoHost <value>     Spnego host for spnego authentication.
          |  --conf <value>           Kyuubi config property pair, formatted key=value.
+         |  -zk, --zk-quorum <value>
+         |                           $zkHelpString
+         |  -n, --namespace <value>  The namespace, using kyuubi-defaults/conf if absent.
+         |  -s, --host <value>       Hostname or IP address of a service.
+         |  -p, --port <value>       Listening port of a service.
          |
          |Command: create [batch|server] [options]
          |${"\t"}Create a resource.
diff --git a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/TestPrematureExit.scala b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/TestPrematureExit.scala
index 6477b9315..43e1454f3 100644
--- a/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/TestPrematureExit.scala
+++ b/kyuubi-ctl/src/test/scala/org/apache/kyuubi/ctl/TestPrematureExit.scala
@@ -99,4 +99,30 @@ trait TestPrematureExit {
         _.getMessage.getFormattedMessage.contains(searchString)))
     }
   }
+
+  /** Returns true if the script exits and the given search string is printed. */
+  private[kyuubi] def testPrematureExitForAdminControlCli(
+      input: Array[String],
+      searchString: String): String = {
+    testPrematureExitForControlCli(input, searchString, AdminControlCli)
+  }
+
+  def testPrematureExitForAdminControlCliArgs(args: Array[String], searchString: String): Unit = {
+    val logAppender = new LogAppender("test premature exit")
+    withLogAppender(logAppender) {
+      val thread = new Thread {
+        override def run(): Unit =
+          try {
+            new AdminControlCliArguments(args)
+          } catch {
+            case e: Exception =>
+              error(e)
+          }
+      }
+      thread.start()
+      thread.join()
+      assert(logAppender.loggingEvents.exists(
+        _.getMessage.getFormattedMessage.contains(searchString)))
+    }
+  }
 }
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java
new file mode 100644
index 000000000..46853bb51
--- /dev/null
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/AdminRestApi.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.client;
+
+public class AdminRestApi {
+  private KyuubiRestClient client;
+
+  private static final String API_BASE_PATH = "admin";
+
+  private AdminRestApi() {}
+
+  public AdminRestApi(KyuubiRestClient client) {
+    this.client = client;
+  }
+
+  public String refreshHadoopConf() {
+    String path = String.format("%s/%s", API_BASE_PATH, "refresh/hadoop_conf");
+    return this.getClient().post(path, null, client.getAuthHeader());
+  }
+
+  private IRestClient getClient() {
+    return this.client.getHttpClient();
+  }
+}
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/RestClient.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/RestClient.java
index 8ad072340..7b93f559e 100644
--- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/RestClient.java
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/RestClient.java
@@ -80,8 +80,10 @@ public class RestClient implements IRestClient {
 
   @Override
   public String post(String path, String body, String authHeader) {
-    RequestBuilder postRequestBuilder =
-        RequestBuilder.post().setEntity(new StringEntity(body, StandardCharsets.UTF_8));
+    RequestBuilder postRequestBuilder = RequestBuilder.post();
+    if (body != null) {
+      postRequestBuilder.setEntity(new StringEntity(body, StandardCharsets.UTF_8));
+    }
     return doRequest(buildURI(path), authHeader, postRequestBuilder);
   }
 
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
index 290a325b0..7e01ca06a 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/AdminResource.scala
@@ -52,6 +52,6 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
     }
     info(s"Reloading the Kyuubi server hadoop conf")
     KyuubiServer.reloadHadoopConf()
-    Response.ok().build()
+    Response.ok(s"Refresh the hadoop conf for ${fe.connectionUrl} successfully.").build()
   }
 }
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala
new file mode 100644
index 000000000..b52c6f67e
--- /dev/null
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminCtlSuite.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.server.rest.client
+
+import org.apache.kyuubi.RestClientTestHelper
+import org.apache.kyuubi.ctl.{CtlConf, TestPrematureExit}
+
+class AdminCtlSuite extends RestClientTestHelper with TestPrematureExit {
+  override def beforeAll(): Unit = {
+    super.beforeAll()
+    System.setProperty(CtlConf.CTL_REST_CLIENT_BASE_URL.key, baseUri.toString)
+    System.setProperty(CtlConf.CTL_REST_CLIENT_SPNEGO_HOST.key, "localhost")
+    System.setProperty(CtlConf.CTL_REST_CLIENT_AUTH_SCHEMA.key, "spnego")
+  }
+
+  override def afterAll(): Unit = {
+    System.clearProperty(CtlConf.CTL_REST_CLIENT_BASE_URL.key)
+    System.clearProperty(CtlConf.CTL_REST_CLIENT_SPNEGO_HOST.key)
+    System.clearProperty(CtlConf.CTL_REST_CLIENT_AUTH_SCHEMA.key)
+    super.afterAll()
+  }
+
+  test("refresh config - hadoop conf") {
+    val args = Array("refresh", "config", "hadoopConf")
+    testPrematureExitForAdminControlCli(
+      args,
+      s"Refresh the hadoop conf for ${fe.connectionUrl} successfully.")
+  }
+}
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala
new file mode 100644
index 000000000..9ba0b89a4
--- /dev/null
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/server/rest/client/AdminRestApiSuite.scala
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kyuubi.server.rest.client
+
+import org.apache.kyuubi.RestClientTestHelper
+import org.apache.kyuubi.client.{AdminRestApi, KyuubiRestClient}
+
+class AdminRestApiSuite extends RestClientTestHelper {
+  test("refresh kyuubi server hadoop conf") {
+    val spnegoKyuubiRestClient: KyuubiRestClient =
+      KyuubiRestClient.builder(baseUri.toString)
+        .authHeaderMethod(KyuubiRestClient.AuthHeaderMethod.SPNEGO)
+        .spnegoHost("localhost")
+        .build()
+    val adminRestApi = new AdminRestApi(spnegoKyuubiRestClient)
+    val result = adminRestApi.refreshHadoopConf()
+    assert(result === s"Refresh the hadoop conf for ${fe.connectionUrl} successfully.")
+  }
+}