You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ch...@apache.org on 2023/02/27 03:09:23 UTC

[kyuubi] branch PR_4002-rebase created (now 367682af9)

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

chengpan pushed a change to branch PR_4002-rebase
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


      at 367682af9 Kyuubi UI draft

This branch includes the following new commits:

     new 367682af9 Kyuubi UI draft

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[kyuubi] 01/01: Kyuubi UI draft

Posted by ch...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch PR_4002-rebase
in repository https://gitbox.apache.org/repos/asf/kyuubi.git

commit 367682af9b671d4a3e3edc728f8eda70daf31ed0
Author: hezhao2 <he...@cisco.com>
AuthorDate: Mon Nov 14 14:36:08 2022 +0800

    Kyuubi UI draft
---
 .rat-excludes                                      |    1 +
 build/Dockerfile                                   |    2 +-
 docs/deployment/settings.md                        |    6 +
 .../engine/spark/SparkTBinaryFrontendService.scala |   27 +-
 .../spark/ZookeeperSparkEngineRegisterSuite.scala  |   44 +
 .../org/apache/kyuubi/config/KyuubiConf.scala      |    7 +
 .../apache/kyuubi/config/KyuubiReservedKeys.scala  |    5 +
 .../org/apache/kyuubi/server/log/ServerLog.scala   |  167 ++++
 .../apache/kyuubi/session/AbstractSession.scala    |    4 +
 .../scala/org/apache/kyuubi/session/Session.scala  |    4 +-
 .../apache/kyuubi/ha/client/DiscoveryClient.scala  |    4 +-
 .../zookeeper/ZookeeperDiscoveryClient.scala       |   57 +-
 .../apache/kyuubi/client/api/v1/dto/Engine.java    |  117 ++-
 .../kyuubi/client/api/v1/dto/HadoopConfData.java   |   59 ++
 .../apache/kyuubi/client/api/v1/dto/SQLDetail.java |  168 ++++
 .../apache/kyuubi/client/api/v1/dto/Server.java    |  148 +++
 .../apache/kyuubi/client/api/v1/dto/ServerLog.java |   74 ++
 kyuubi-server/pom.xml                              |   11 +
 .../kyuubi/events/KyuubiOperationEvent.scala       |   15 +-
 .../apache/kyuubi/events/KyuubiSessionEvent.scala  |    8 +-
 .../kyuubi/server/KyuubiRestFrontendService.scala  |   19 +-
 .../org/apache/kyuubi/server/KyuubiServer.scala    |   12 +
 .../server/KyuubiTBinaryFrontendService.scala      |    4 +
 .../kyuubi/server/api/WebUIProxyServlet.scala      |   81 ++
 .../kyuubi/server/api/v1/AdminResource.scala       |  192 +++-
 .../kyuubi/server/api/v1/ApiRootResource.scala     |   28 +-
 .../kyuubi/server/api/v1/OperationsResource.scala  |  114 ++-
 .../kyuubi/server/api/v1/SessionsResource.scala    |  119 ++-
 .../apache/kyuubi/session/KyuubiSessionImpl.scala  |   14 +-
 .../scala/org/apache/kyuubi/util/OSUtils.scala}    |   25 +-
 kyuubi-server/web-ui/package.json                  |    1 +
 kyuubi-server/web-ui/pnpm-lock.yaml                | 1045 ++++++++++----------
 .../src/{locales/en_US => api/server}/index.ts     |   33 +-
 .../en_US/index.ts => api/server/types.ts}         |   12 +-
 kyuubi-server/web-ui/src/api/session/index.ts      |   76 ++
 .../en_US/index.ts => api/session/types.ts}        |   13 +-
 kyuubi-server/web-ui/src/components/menu/index.vue |   32 +-
 .../web-ui/src/components/modal/index.vue          |  112 +++
 kyuubi-server/web-ui/src/locales/en_US/index.ts    |   41 +-
 kyuubi-server/web-ui/src/locales/zh_CN/index.ts    |   43 +-
 kyuubi-server/web-ui/src/router/index.ts           |   15 +-
 .../src/router/{contact => run-sql}/index.ts       |    6 +-
 .../src/router/{operation => server}/index.ts      |   12 +-
 .../src/router/{workload => session}/index.ts      |   23 +-
 kyuubi-server/web-ui/src/styles/element/index.scss |    9 +-
 kyuubi-server/web-ui/src/styles/index.scss         |   10 +
 kyuubi-server/web-ui/src/utils/index.ts            |   24 +
 kyuubi-server/web-ui/src/utils/request.ts          |    2 +-
 .../views/common/use-table.ts}                     |   43 +-
 .../src/views/layout/components/aside/index.vue    |   62 +-
 .../src/views/layout/components/aside/types.ts     |   71 --
 kyuubi-server/web-ui/src/views/layout/index.vue    |    7 +-
 .../src/views/operation/completedJobs/index.vue    |   29 -
 .../src/views/operation/runningJobs/index.vue      |   29 -
 .../src/views/{contact => run-sql}/index.vue       |    8 +-
 .../web-ui/src/views/server/engine/index.vue       |  122 +++
 .../src/views/server/kyuubi-service/index.vue      |   79 ++
 .../web-ui/src/views/session/operation/index.vue   |  261 +++++
 .../src/views/session/session-statistics/index.vue |  172 ++++
 .../src/views/session/sql-statistics/index.vue     |  203 ++++
 .../web-ui/src/views/workload/analysis/index.vue   |   29 -
 .../web-ui/src/views/workload/query/index.vue      |   29 -
 .../web-ui/src/views/workload/queue/index.vue      |   29 -
 .../web-ui/src/views/workload/session/index.vue    |   29 -
 kyuubi-server/web-ui/vite.config.ts                |    9 +
 pom.xml                                            |    6 +
 66 files changed, 3314 insertions(+), 948 deletions(-)

diff --git a/.rat-excludes b/.rat-excludes
index 7a841cf9c..fa47484e8 100644
--- a/.rat-excludes
+++ b/.rat-excludes
@@ -52,6 +52,7 @@ build/scala-*/**
 **/*.derby.sql
 **/*.mysql.sql
 **/node/**
+**/web-ui/**
 **/web-ui/dist/**
 **/pnpm-lock.yaml
 **/node_modules/**
diff --git a/build/Dockerfile b/build/Dockerfile
index 8ecc6c8b7..8a99465ad 100644
--- a/build/Dockerfile
+++ b/build/Dockerfile
@@ -76,7 +76,7 @@ RUN set -ex && \
     ln -snf /bin/bash /bin/sh && \
     useradd -u ${kyuubi_uid} -g root kyuubi && \
     mkdir -p ${KYUUBI_HOME} ${KYUUBI_LOG_DIR} ${KYUUBI_PID_DIR} ${KYUUBI_WORK_DIR_ROOT} && \
-    chmod ug+rw -R ${KYUUBI_HOME} && \
+    chmod ug+rwx -R ${KYUUBI_HOME} && \
     chmod a+rwx -R ${KYUUBI_WORK_DIR_ROOT} && \
     rm -rf /var/cache/apt/*
 
diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index dac72825e..a066dcd64 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -447,6 +447,12 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
 | kyuubi.operation.spark.listener.enabled         | true                                                                            | When set to true, Spark engine registers an SQLOperationListener before executing the statement, logging a few summary statistics when each stage completes.                                                                                                                                                                                                           [...]
 | kyuubi.operation.status.polling.timeout         | PT5S                                                                            | Timeout(ms) for long polling asynchronous running sql query's status                                                                                                                                                                                                                                                                                                   [...]
 
+### Proxy
+
+|              Key              | Default |                    Meaning                    |  Type   | Since |
+|-------------------------------|---------|-----------------------------------------------|---------|-------|
+| kyuubi.proxy.spark.ui.enabled | false   | Whether to enabled kyuubi help proxy spark ui | boolean | 1.7.0 |
+
 ### Server
 
 |                           Key                            |      Default      |                                                                                                                    Meaning                                                                                                                     |   Type   | Since |
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkTBinaryFrontendService.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkTBinaryFrontendService.scala
index d4eaf3454..b36cac071 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkTBinaryFrontendService.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkTBinaryFrontendService.scala
@@ -18,6 +18,7 @@
 package org.apache.kyuubi.engine.spark
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable
 
 import org.apache.hadoop.io.Text
 import org.apache.hadoop.security.{Credentials, UserGroupInformation}
@@ -94,13 +95,37 @@ class SparkTBinaryFrontendService(
   }
 
   override def attributes: Map[String, String] = {
-    Map(KYUUBI_ENGINE_ID -> KyuubiSparkUtil.engineId)
+    val settings = sc.getConf.getAll.toMap
+    val memory = sc.getExecutorMemoryStatus.size + "*" +
+      settings.get(SPARK_ENGINE_EXECUTOR_MEMORY).orNull
+
+    val cores = sc.getExecutorMemoryStatus.size + "*" +
+      settings.get(SPARK_ENGINE_EXECUTOR_CORES).orNull
+    val address = java.net.InetAddress.getLocalHost.getHostAddress
+
+    val attr = new mutable.HashMap[String, String]
+    attr.put(KYUUBI_ENGINE_ID, KyuubiSparkUtil.engineId)
+    attr.put(KYUUBI_ENGINE_SUBMIT_TIME_KEY, sc.startTime.toString)
+    attr.put(KYUUBI_ENGINE_MEMORY, memory)
+    attr.put(KYUUBI_ENGINE_CPU, cores)
+    attr.put(KYUUBI_ENGINE_UI_PORT, sc.getConf.get("spark.ui.port"))
+    attr.put(KYUUBI_ENGINE_DRIVER_IP, address)
+    sc.uiWebUrl.foreach { ui => attr.put(KYUUBI_ENGINE_URL, ui.replace("//", "")) }
+    attr.toMap
   }
 }
 
 object SparkTBinaryFrontendService extends Logging {
 
   val HIVE_DELEGATION_TOKEN = new Text("HIVE_DELEGATION_TOKEN")
+  final val SPARK_ENGINE_DRIVER_MEMORY = "spark.driver.memory"
+  final val SPARK_ENGINE_EXECUTOR_MEMORY = "spark.executor.memory"
+
+  final val SPARK_ENGINE_EXECUTOR_CORES = "spark.executor.cores"
+  final val SPARK_ENGINE_DRIVER_CORES = "spark.driver.cores"
+
+  final val SPARK_ENGINE_EXECUTOR_INSTANCE = "spark.executor.instances"
+  final val SPARK_ENGINE_EXECUTOR_MAX_INSTANCE = "spark.dynamicAllocation.maxExecutors"
 
   private[spark] def renewDelegationToken(sc: SparkContext, delegationToken: String): Unit = {
     val newCreds = KyuubiHadoopUtils.decodeCredentials(delegationToken)
diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/ZookeeperSparkEngineRegisterSuite.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/ZookeeperSparkEngineRegisterSuite.scala
new file mode 100644
index 000000000..35c2c31b9
--- /dev/null
+++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/ZookeeperSparkEngineRegisterSuite.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.engine.spark
+
+import java.nio.charset.StandardCharsets
+import java.util.UUID
+
+import org.apache.kyuubi.config.KyuubiConf.PROXY_SPARK_UI_ENABLED
+import org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider
+
+class ZookeeperSparkEngineRegisterSuite extends WithDiscoverySparkSQLEngine
+  with WithEmbeddedZookeeper {
+  override def withKyuubiConf: Map[String, String] =
+    super.withKyuubiConf ++ zookeeperConf ++ Map(
+      PROXY_SPARK_UI_ENABLED.key -> "true",
+      "spark.ui.enabled" -> "true")
+
+  override val namespace: String = s"/kyuubi/deregister_test/${UUID.randomUUID}"
+
+  test("Spark Engine Register Zookeeper with spark ui info") {
+    zookeeperConf.foreach(entry => kyuubiConf.set(entry._1, entry._2))
+    val client = ZookeeperClientProvider.buildZookeeperClient(kyuubiConf)
+    client.start()
+    val bytes =
+      client.getData.forPath(namespace + "/" + client.getChildren.forPath(namespace).get(0))
+    val data = new String(bytes, StandardCharsets.UTF_8)
+    assert(data.contains("spark.ui"))
+  }
+}
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 517d92db3..04a9bc498 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
@@ -365,6 +365,13 @@ object KyuubiConf {
       .booleanConf
       .createWithDefault(true)
 
+  val PROXY_SPARK_UI_ENABLED: ConfigEntry[Boolean] =
+    buildConf("kyuubi.proxy.spark.ui.enabled")
+      .doc("Whether to enabled kyuubi help proxy spark ui")
+      .version("1.7.0")
+      .booleanConf
+      .createWithDefault(false)
+
   // ///////////////////////////////////////////////////////////////////////////////////////////////
   //                              Frontend Service Configuration                                 //
   // ///////////////////////////////////////////////////////////////////////////////////////////////
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiReservedKeys.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiReservedKeys.scala
index 8b42e659f..ef2db7067 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiReservedKeys.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiReservedKeys.scala
@@ -32,6 +32,7 @@ object KyuubiReservedKeys {
   final val KYUUBI_ENGINE_NAME = "kyuubi.engine.name"
   final val KYUUBI_ENGINE_URL = "kyuubi.engine.url"
   final val KYUUBI_ENGINE_SUBMIT_TIME_KEY = "kyuubi.engine.submit.time"
+  final val KYUUBI_SERVER_SUBMIT_TIME = "kyuubi.server.submit.time"
   final val KYUUBI_ENGINE_CREDENTIALS_KEY = "kyuubi.engine.credentials"
   final val KYUUBI_SESSION_HANDLE_KEY = "kyuubi.session.handle"
   final val KYUUBI_SESSION_ENGINE_LAUNCH_HANDLE_GUID =
@@ -43,4 +44,8 @@ object KyuubiReservedKeys {
   final val KYUUBI_OPERATION_SET_CURRENT_DATABASE = "kyuubi.operation.set.current.database"
   final val KYUUBI_OPERATION_GET_CURRENT_DATABASE = "kyuubi.operation.get.current.database"
   final val KYUUBI_OPERATION_HANDLE_KEY = "kyuubi.operation.handle"
+  final val KYUUBI_ENGINE_MEMORY = "kyuubi.engine.memory"
+  final val KYUUBI_ENGINE_CPU = "kyuubi.engine.cpu"
+  final val KYUUBI_ENGINE_DRIVER_IP = "kyuubi.engine.driver.ip"
+  final val KYUUBI_ENGINE_UI_PORT = "kyuubi.engine.ui.port"
 }
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/server/log/ServerLog.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/server/log/ServerLog.scala
new file mode 100644
index 000000000..3b555bbf6
--- /dev/null
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/server/log/ServerLog.scala
@@ -0,0 +1,167 @@
+/*
+ * 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.log
+
+import java.io.{BufferedReader, IOException}
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Path, Paths}
+import java.util.{ArrayList => JArrayList}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.kyuubi.{KyuubiException, Logging}
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.config.KyuubiConf.SERVER_OPERATION_LOG_DIR_ROOT
+import org.apache.kyuubi.operation.log.SeekableBufferedReader
+
+object ServerLog extends Logging {
+  final private val SERVER_LOG: InheritableThreadLocal[ServerLog] = {
+    new InheritableThreadLocal[ServerLog] {
+      override def initialValue(): ServerLog = null
+    }
+  }
+
+  def setCurrentServerLog(serverLog: ServerLog): Unit = {
+    SERVER_LOG.set(serverLog)
+  }
+
+  def getCurrentServerLog: ServerLog = SERVER_LOG.get()
+
+  def removeCurrentServerLog(): Unit = SERVER_LOG.remove()
+
+  def createServerLog(conf: KyuubiConf): ServerLog = {
+    try {
+      val logPath = conf.get(SERVER_OPERATION_LOG_DIR_ROOT)
+      val logFile = Paths.get(logPath)
+      info(s"Creating server log $logFile")
+      new ServerLog(logFile)
+    } catch {
+      case e: IOException =>
+        error(s"Failed to create server log", e)
+        null
+    }
+  }
+
+}
+
+class ServerLog(path: Path) {
+
+  private lazy val writer = Files.newBufferedWriter(path, StandardCharsets.UTF_8)
+  private lazy val reader = Files.newBufferedReader(path, StandardCharsets.UTF_8)
+
+  private lazy val extraPaths: ListBuffer[Path] = ListBuffer()
+  private lazy val extraReaders: ListBuffer[BufferedReader] = ListBuffer()
+  private var lastSeekReadPos = 0
+  private var seekableReader: SeekableBufferedReader = _
+
+  def addExtraLog(path: Path): Unit = synchronized {
+    try {
+      extraReaders += Files.newBufferedReader(path, StandardCharsets.UTF_8)
+      extraPaths += path
+      Option(seekableReader).foreach(_.close)
+      seekableReader = null
+    } catch {
+      case _: IOException =>
+    }
+  }
+
+  private def readLogs(
+      reader: BufferedReader,
+      lastRows: Int,
+      maxRows: Int): (JArrayList[String], Int) = {
+    val logs = new JArrayList[String]
+    var i = 0
+    try {
+      var line: String = reader.readLine()
+      while ((i < lastRows || maxRows <= 0) && line != null) {
+        logs.add(line)
+        line = reader.readLine()
+        i += 1
+      }
+      (logs, i)
+    } catch {
+      case e: IOException =>
+        val absPath = path.toAbsolutePath
+        val opHandle = absPath.getFileName
+        throw new KyuubiException(s"Operation[$opHandle] log file $absPath is not found", e)
+    }
+  }
+
+  /**
+   * Read to log file line by line
+   *
+   * @param maxRows maximum result number can reach
+   */
+  def read(maxRows: Int): JArrayList[String] = synchronized {
+    val (logs, lines) = readLogs(reader, maxRows, maxRows)
+    var lastRows = maxRows - lines
+    for (extraReader <- extraReaders if lastRows > 0 || maxRows <= 0) {
+      val (extraLogs, extraRows) = readLogs(extraReader, lastRows, maxRows)
+      lastRows = lastRows - extraRows
+      logs.addAll(extraLogs)
+    }
+    logs
+  }
+
+  def close(): Unit = synchronized {
+    closeExtraReaders()
+
+    trySafely {
+      reader.close()
+    }
+    trySafely {
+      writer.close()
+    }
+
+    if (seekableReader != null) {
+      lastSeekReadPos = 0
+      trySafely {
+        seekableReader.close()
+      }
+    }
+
+    trySafely {
+      Files.delete(path)
+    }
+  }
+
+  private def trySafely(f: => Unit): Unit = {
+    try {
+      f
+    } catch {
+      case e: IOException =>
+        // Printing log here may cause a deadlock. The lock order of OperationLog.write
+        // is RootLogger -> LogDivertAppender -> OperationLog. If printing log here, the
+        // lock order is OperationLog -> RootLogger. So the exception is thrown and
+        // processing at the invocations
+        throw new IOException(
+          s"Failed to remove corresponding log file of operation: ${path.toAbsolutePath}",
+          e)
+    }
+  }
+
+  private def closeExtraReaders(): Unit = {
+    extraReaders.foreach { extraReader =>
+      try {
+        extraReader.close()
+      } catch {
+        case _: IOException => // for the outside log file reader, ignore it
+      }
+    }
+  }
+}
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/session/AbstractSession.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/session/AbstractSession.scala
index 1a8c51ccd..25254e2fb 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/session/AbstractSession.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/session/AbstractSession.scala
@@ -61,6 +61,10 @@ abstract class AbstractSession(
 
   final private val opHandleSet = new java.util.HashSet[OperationHandle]
 
+  override def allOperations(): Iterable[OperationHandle] = opHandleSet.asScala
+
+  override def getOperationCount: Long = opHandleSet.size()
+
   private def acquire(userAccess: Boolean): Unit = synchronized {
     if (userAccess) {
       _lastAccessTime = System.currentTimeMillis
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/session/Session.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/session/Session.scala
index bc9f9a8f6..6c37af69a 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/session/Session.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/session/Session.scala
@@ -17,7 +17,7 @@
 
 package org.apache.kyuubi.session
 
-import org.apache.hive.service.rpc.thrift.{TGetInfoType, TGetInfoValue, TGetResultSetMetadataResp, TProtocolVersion, TRowSet}
+import org.apache.hive.service.rpc.thrift._
 
 import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation
 import org.apache.kyuubi.operation.OperationHandle
@@ -39,6 +39,8 @@ trait Session {
   def lastIdleTime: Long
   def getNoOperationTime: Long
   def sessionIdleTimeoutThreshold: Long
+  def allOperations(): Iterable[OperationHandle]
+  def getOperationCount(): Long
 
   def sessionManager: SessionManager
 
diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/DiscoveryClient.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/DiscoveryClient.scala
index 77588180e..86ddd678d 100644
--- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/DiscoveryClient.scala
+++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/DiscoveryClient.scala
@@ -197,7 +197,9 @@ object DiscoveryClient {
         maybeInfos("hive.server2.thrift.port").toInt)
     } else {
       val strings = instance.split(":")
-      (strings(0), strings(1).toInt)
+      val host: String = strings(0)
+      val port: Int = if (strings.length == 1) 0 else strings(1).toInt
+      (host, port)
     }
   }
 }
diff --git a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/zookeeper/ZookeeperDiscoveryClient.scala b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/zookeeper/ZookeeperDiscoveryClient.scala
index daa27047e..53639525e 100644
--- a/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/zookeeper/ZookeeperDiscoveryClient.scala
+++ b/kyuubi-ha/src/main/scala/org/apache/kyuubi/ha/client/zookeeper/ZookeeperDiscoveryClient.scala
@@ -29,34 +29,21 @@ import org.apache.curator.framework.CuratorFramework
 import org.apache.curator.framework.recipes.atomic.{AtomicValue, DistributedAtomicInteger}
 import org.apache.curator.framework.recipes.locks.InterProcessSemaphoreMutex
 import org.apache.curator.framework.recipes.nodes.PersistentNode
-import org.apache.curator.framework.state.ConnectionState
-import org.apache.curator.framework.state.ConnectionState.CONNECTED
-import org.apache.curator.framework.state.ConnectionState.LOST
-import org.apache.curator.framework.state.ConnectionState.RECONNECTED
-import org.apache.curator.framework.state.ConnectionStateListener
+import org.apache.curator.framework.state.{ConnectionState, ConnectionStateListener}
+import org.apache.curator.framework.state.ConnectionState.{CONNECTED, LOST, RECONNECTED}
 import org.apache.curator.retry.RetryForever
 import org.apache.curator.utils.ZKPaths
-import org.apache.zookeeper.CreateMode
+import org.apache.zookeeper.{CreateMode, KeeperException, WatchedEvent, Watcher}
 import org.apache.zookeeper.CreateMode.PERSISTENT
-import org.apache.zookeeper.KeeperException
 import org.apache.zookeeper.KeeperException.NodeExistsException
-import org.apache.zookeeper.WatchedEvent
-import org.apache.zookeeper.Watcher
 
-import org.apache.kyuubi.KYUUBI_VERSION
-import org.apache.kyuubi.KyuubiException
-import org.apache.kyuubi.KyuubiSQLException
-import org.apache.kyuubi.Logging
+import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiException, KyuubiSQLException, Logging}
 import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.config.KyuubiConf.PROXY_SPARK_UI_ENABLED
 import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_ID
-import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ENGINE_REF_ID
-import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ZK_NODE_TIMEOUT
-import org.apache.kyuubi.ha.HighAvailabilityConf.HA_ZK_PUBLISH_CONFIGS
-import org.apache.kyuubi.ha.client.DiscoveryClient
-import org.apache.kyuubi.ha.client.ServiceDiscovery
-import org.apache.kyuubi.ha.client.ServiceNodeInfo
-import org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider.buildZookeeperClient
-import org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider.getGracefulStopThreadDelay
+import org.apache.kyuubi.ha.HighAvailabilityConf.{HA_ENGINE_REF_ID, HA_ZK_NODE_TIMEOUT, HA_ZK_PUBLISH_CONFIGS}
+import org.apache.kyuubi.ha.client.{DiscoveryClient, ServiceDiscovery, ServiceNodeInfo}
+import org.apache.kyuubi.ha.client.zookeeper.ZookeeperClientProvider.{buildZookeeperClient, getGracefulStopThreadDelay}
 import org.apache.kyuubi.ha.client.zookeeper.ZookeeperDiscoveryClient.connectionChecker
 import org.apache.kyuubi.util.ThreadUtils
 
@@ -215,16 +202,23 @@ class ZookeeperDiscoveryClient(conf: KyuubiConf) extends DiscoveryClient {
       val size = sizeOpt.getOrElse(hosts.size())
       hosts.asScala.takeRight(size).map { p =>
         val path = ZKPaths.makePath(namespace, p)
-        val instance = new String(zkClient.getData.forPath(path), StandardCharsets.UTF_8)
+        // zookeeper node data may contains public conf or engine ui info
+        val znodeData = new String(zkClient.getData.forPath(path), StandardCharsets.UTF_8)
+        val instance = znodeData.split(";")(0)
+        val ui =
+          znodeData.split(";").find(_.startsWith("spark.ui=")).map(_.stripPrefix("spark.ui="))
         val (host, port) = DiscoveryClient.parseInstanceHostPort(instance)
-        val attributes =
-          p.split(";").map(_.split("=", 2)).filter(_.length == 2).map(kv =>
-            (kv.head, kv.last)).toMap
-        val version = attributes.get("version")
-        val engineRefId = attributes.get("refId")
+        var attributes =
+          p.split(";").map(_.split("=", 2)).filter(_.size == 2).map(kv => (kv.head, kv.last)).toMap
+        val version = p.split(";").find(_.startsWith("version=")).map(_.stripPrefix("version="))
+        val engineRefId = p.split(";").find(_.startsWith("refId=")).map(_.stripPrefix("refId="))
         val engineIdStr = attributes.get(KYUUBI_ENGINE_ID).map(" engine id:" + _).getOrElse("")
         info(s"Get service instance:$instance$engineIdStr and version:${version.getOrElse("")} " +
           s"under $namespace")
+        attributes = ui match {
+          case Some(_ui) => attributes + ("ui" -> _ui)
+          case None => attributes
+        }
         ServiceNodeInfo(namespace, p, host, port, version, engineRefId, attributes)
       }
     } catch {
@@ -377,12 +371,19 @@ class ZookeeperDiscoveryClient(conf: KyuubiConf) extends DiscoveryClient {
     val createMode =
       if (external) CreateMode.PERSISTENT_SEQUENTIAL
       else CreateMode.EPHEMERAL_SEQUENTIAL
-    val znodeData =
+    var znodeData =
       if (conf.get(HA_ZK_PUBLISH_CONFIGS) && session.isEmpty) {
         addConfsToPublish(conf, instance)
       } else {
         instance
       }
+    val ui = attributes.get("ui")
+    znodeData =
+      if (conf.get(PROXY_SPARK_UI_ENABLED) && ui.nonEmpty) {
+        znodeData + ";spark.ui=" + ui.get
+      } else {
+        znodeData
+      }
     try {
       localServiceNode = new PersistentNode(
         zkClient,
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Engine.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Engine.java
index 133897dd6..6b6141868 100644
--- a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Engine.java
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Engine.java
@@ -17,7 +17,6 @@
 
 package org.apache.kyuubi.client.api.v1.dto;
 
-import java.util.Collections;
 import java.util.Map;
 import java.util.Objects;
 import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
@@ -34,7 +33,17 @@ public class Engine {
   private String namespace;
   private Map<String, String> attributes;
 
-  public Engine() {}
+  private String createTime;
+  private String url;
+  private String host;
+  private int port;
+
+  private String id;
+  private String status;
+
+  private String memoryTotal;
+
+  private String cpuTotal;
 
   public Engine(
       String version,
@@ -44,7 +53,15 @@ public class Engine {
       String subdomain,
       String instance,
       String namespace,
-      Map<String, String> attributes) {
+      Map<String, String> attributes,
+      String createTime,
+      String url,
+      String host,
+      int port,
+      String id,
+      String status,
+      String memoryTotal,
+      String cpuTotal) {
     this.version = version;
     this.user = user;
     this.engineType = engineType;
@@ -53,6 +70,14 @@ public class Engine {
     this.instance = instance;
     this.namespace = namespace;
     this.attributes = attributes;
+    this.createTime = createTime;
+    this.url = url;
+    this.host = host;
+    this.port = port;
+    this.id = id;
+    this.status = status;
+    this.memoryTotal = memoryTotal;
+    this.cpuTotal = cpuTotal;
   }
 
   public String getVersion() {
@@ -112,9 +137,6 @@ public class Engine {
   }
 
   public Map<String, String> getAttributes() {
-    if (null == attributes) {
-      return Collections.emptyMap();
-    }
     return attributes;
   }
 
@@ -122,6 +144,70 @@ public class Engine {
     this.attributes = attributes;
   }
 
+  public String getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(String createTime) {
+    this.createTime = createTime;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  public void setUrl(String url) {
+    this.url = url;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public void setHost(String host) {
+    this.host = host;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public String getId() {
+    return id;
+  }
+
+  public void setId(String id) {
+    this.id = id;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  public String getMemoryTotal() {
+    return memoryTotal;
+  }
+
+  public void setMemoryTotal(String memoryTotal) {
+    this.memoryTotal = memoryTotal;
+  }
+
+  public String getCpuTotal() {
+    return cpuTotal;
+  }
+
+  public void setCpuTotal(String cpuTotal) {
+    this.cpuTotal = cpuTotal;
+  }
+
   @Override
   public boolean equals(Object o) {
     if (this == o) return true;
@@ -134,7 +220,14 @@ public class Engine {
         && Objects.equals(getSubdomain(), that.getSubdomain())
         && Objects.equals(getInstance(), that.getInstance())
         && Objects.equals(getNamespace(), that.getNamespace())
-        && Objects.equals(getAttributes(), that.getAttributes());
+        && Objects.equals(getAttributes(), that.getAttributes())
+        && Objects.equals(getCreateTime(), that.getCreateTime())
+        && Objects.equals(getUrl(), that.getUrl())
+        && Objects.equals(getHost(), that.getHost())
+        && Objects.equals(getPort(), that.getPort())
+        && Objects.equals(getStatus(), that.getStatus())
+        && Objects.equals(getCpuTotal(), that.getCpuTotal())
+        && Objects.equals(getMemoryTotal(), that.getMemoryTotal());
   }
 
   @Override
@@ -147,7 +240,15 @@ public class Engine {
         getSubdomain(),
         getInstance(),
         getNamespace(),
-        getAttributes());
+        getAttributes(),
+        getCreateTime(),
+        getUrl(),
+        getHost(),
+        getPort(),
+        getId(),
+        getStatus(),
+        getCpuTotal(),
+        getMemoryTotal());
   }
 
   @Override
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/HadoopConfData.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/HadoopConfData.java
new file mode 100644
index 000000000..bd9db81b3
--- /dev/null
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/HadoopConfData.java
@@ -0,0 +1,59 @@
+/*
+ * 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.api.v1.dto;
+
+import java.util.Objects;
+
+public class HadoopConfData {
+  private String key;
+  private String value;
+
+  public HadoopConfData(String key, String value) {
+    this.key = key;
+    this.value = value;
+  }
+
+  public String getKey() {
+    return key;
+  }
+
+  public void setKey(String key) {
+    this.key = key;
+  }
+
+  public String getValue() {
+    return value;
+  }
+
+  public void setValue(String value) {
+    this.value = value;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    HadoopConfData that = (HadoopConfData) o;
+    return Objects.equals(key, that.key) && Objects.equals(value, that.value);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(key, value);
+  }
+}
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SQLDetail.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SQLDetail.java
new file mode 100644
index 000000000..f436a3509
--- /dev/null
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/SQLDetail.java
@@ -0,0 +1,168 @@
+/*
+ * 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.api.v1.dto;
+
+import java.util.Objects;
+
+public class SQLDetail {
+  private String sessionId;
+  private String sessionUser;
+  private String statementId;
+  private Long createTime;
+  private Long completeTime;
+  private String statement;
+  private String engineId;
+  private String engineType;
+  private String engineShareLevel;
+  private String exception;
+
+  public SQLDetail(
+      String sessionId,
+      String sessionUser,
+      String statementId,
+      Long createTime,
+      Long completeTime,
+      String statement,
+      String engineId,
+      String engineType,
+      String engineShareLevel,
+      String exception) {
+    this.sessionId = sessionId;
+    this.sessionUser = sessionUser;
+    this.statementId = statementId;
+    this.createTime = createTime;
+    this.completeTime = completeTime;
+    this.statement = statement;
+    this.engineId = engineId;
+    this.engineType = engineType;
+    this.engineShareLevel = engineShareLevel;
+    this.exception = exception;
+  }
+
+  public String getSessionId() {
+    return sessionId;
+  }
+
+  public void setSessionId(String sessionId) {
+    this.sessionId = sessionId;
+  }
+
+  public String getSessionUser() {
+    return sessionUser;
+  }
+
+  public void setSessionUser(String sessionUser) {
+    this.sessionUser = sessionUser;
+  }
+
+  public String getStatementId() {
+    return statementId;
+  }
+
+  public void setStatementId(String statementId) {
+    this.statementId = statementId;
+  }
+
+  public Long getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(Long createTime) {
+    this.createTime = createTime;
+  }
+
+  public Long getCompleteTime() {
+    return completeTime;
+  }
+
+  public void setCompleteTime(Long completeTime) {
+    this.completeTime = completeTime;
+  }
+
+  public String getStatement() {
+    return statement;
+  }
+
+  public void setStatement(String statement) {
+    this.statement = statement;
+  }
+
+  public String getEngineId() {
+    return engineId;
+  }
+
+  public void setEngineId(String engineName) {
+    this.engineId = engineName;
+  }
+
+  public String getEngineType() {
+    return engineType;
+  }
+
+  public void setEngineType(String engineType) {
+    this.engineType = engineType;
+  }
+
+  public String getEngineShareLevel() {
+    return engineShareLevel;
+  }
+
+  public void setEngineShareLevel(String engineShareLevel) {
+    this.engineShareLevel = engineShareLevel;
+  }
+
+  public String getException() {
+    return exception;
+  }
+
+  public void setException(String exception) {
+    this.exception = exception;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    SQLDetail that = (SQLDetail) o;
+    return Objects.equals(sessionId, that.sessionId)
+        && Objects.equals(sessionUser, that.sessionUser)
+        && Objects.equals(statementId, that.statementId)
+        && Objects.equals(createTime, that.createTime)
+        && Objects.equals(completeTime, that.completeTime)
+        && Objects.equals(statement, that.statement)
+        && Objects.equals(engineId, that.engineId)
+        && Objects.equals(engineType, that.engineType)
+        && Objects.equals(engineShareLevel, that.engineShareLevel)
+        && Objects.equals(exception, that.exception);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(
+        sessionId,
+        sessionUser,
+        statementId,
+        createTime,
+        completeTime,
+        statement,
+        engineId,
+        engineType,
+        engineShareLevel,
+        exception);
+  }
+}
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Server.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Server.java
new file mode 100644
index 000000000..dbd718651
--- /dev/null
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/Server.java
@@ -0,0 +1,148 @@
+/*
+ * 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.api.v1.dto;
+
+import java.util.Objects;
+
+public class Server {
+
+  private String nodeName;
+  private String namespace;
+  private String instance;
+  private String host;
+  private int port;
+  private Long createTime;
+  private Long memoryTotal;
+  private int cpuTotal;
+  private String status;
+
+  public Server(
+      String nodeName,
+      String namespace,
+      String instance,
+      String host,
+      int port,
+      Long createTime,
+      Long memoryTotal,
+      int cpuTotal,
+      String status) {
+    this.nodeName = nodeName;
+    this.namespace = namespace;
+    this.instance = instance;
+    this.host = host;
+    this.port = port;
+    this.createTime = createTime;
+    this.memoryTotal = memoryTotal;
+    this.cpuTotal = cpuTotal;
+    this.status = status;
+  }
+
+  public String getNodeName() {
+    return nodeName;
+  }
+
+  public void setNodeName(String nodeName) {
+    this.nodeName = nodeName;
+  }
+
+  public String getNamespace() {
+    return namespace;
+  }
+
+  public void setNamespace(String namespace) {
+    this.namespace = namespace;
+  }
+
+  public String getInstance() {
+    return instance;
+  }
+
+  public void setInstance(String instance) {
+    this.instance = instance;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public void setHost(String host) {
+    this.host = host;
+  }
+
+  public int getPort() {
+    return port;
+  }
+
+  public void setPort(int port) {
+    this.port = port;
+  }
+
+  public Long getCreateTime() {
+    return createTime;
+  }
+
+  public void setCreateTime(Long createTime) {
+    this.createTime = createTime;
+  }
+
+  public Long getMemoryTotal() {
+    return memoryTotal;
+  }
+
+  public void setMemoryTotal(Long memoryTotal) {
+    this.memoryTotal = memoryTotal;
+  }
+
+  public int getCpuTotal() {
+    return cpuTotal;
+  }
+
+  public void setCpuTotal(int cpuTotal) {
+    this.cpuTotal = cpuTotal;
+  }
+
+  public String getStatus() {
+    return status;
+  }
+
+  public void setStatus(String status) {
+    this.status = status;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    Server server = (Server) o;
+    return port == server.port
+        && cpuTotal == server.cpuTotal
+        && Objects.equals(nodeName, server.nodeName)
+        && Objects.equals(namespace, server.namespace)
+        && Objects.equals(instance, server.instance)
+        && Objects.equals(host, server.host)
+        && Objects.equals(createTime, server.createTime)
+        && Objects.equals(memoryTotal, server.memoryTotal)
+        && Objects.equals(status, server.status);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(
+        nodeName, namespace, instance, host, port, createTime, memoryTotal, cpuTotal, status);
+  }
+}
diff --git a/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/ServerLog.java b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/ServerLog.java
new file mode 100644
index 000000000..2c3f748f3
--- /dev/null
+++ b/kyuubi-rest-client/src/main/java/org/apache/kyuubi/client/api/v1/dto/ServerLog.java
@@ -0,0 +1,74 @@
+/*
+ * 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.api.v1.dto;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Objects;
+import org.apache.commons.lang3.builder.ReflectionToStringBuilder;
+import org.apache.commons.lang3.builder.ToStringStyle;
+
+public class ServerLog {
+  private List<String> logRowSet;
+  private int rowCount;
+
+  public ServerLog() {}
+
+  public ServerLog(List<String> logRowSet, int rowCount) {
+    this.logRowSet = logRowSet;
+    this.rowCount = rowCount;
+  }
+
+  public List<String> getLogRowSet() {
+    if (null == logRowSet) {
+      return Collections.emptyList();
+    }
+    return logRowSet;
+  }
+
+  public void setLogRowSet(List<String> logRowSet) {
+    this.logRowSet = logRowSet;
+  }
+
+  public int getRowCount() {
+    return rowCount;
+  }
+
+  public void setRowCount(int rowCount) {
+    this.rowCount = rowCount;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    ServerLog that = (ServerLog) o;
+    return getRowCount() == that.getRowCount()
+        && Objects.equals(getLogRowSet(), that.getLogRowSet());
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(getLogRowSet(), getRowCount());
+  }
+
+  @Override
+  public String toString() {
+    return ReflectionToStringBuilder.toString(this, ToStringStyle.JSON_STYLE);
+  }
+}
diff --git a/kyuubi-server/pom.xml b/kyuubi-server/pom.xml
index aebce5dda..08c1b7fd1 100644
--- a/kyuubi-server/pom.xml
+++ b/kyuubi-server/pom.xml
@@ -196,6 +196,11 @@
             <artifactId>jakarta.servlet-api</artifactId>
         </dependency>
 
+        <dependency>
+            <groupId>org.eclipse.jetty</groupId>
+            <artifactId>jetty-proxy</artifactId>
+        </dependency>
+
         <dependency>
             <groupId>org.glassfish.jersey.core</groupId>
             <artifactId>jersey-server</artifactId>
@@ -511,6 +516,12 @@
             <artifactId>flexmark-all</artifactId>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.github.oshi</groupId>
+            <artifactId>oshi-core</artifactId>
+            <version>6.3.2</version>
+        </dependency>
     </dependencies>
 
     <build>
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiOperationEvent.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiOperationEvent.scala
index 74a3a3fad..de939b8fa 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiOperationEvent.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiOperationEvent.scala
@@ -18,8 +18,9 @@
 package org.apache.kyuubi.events
 
 import org.apache.kyuubi.Utils
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_SHARE_LEVEL, ENGINE_TYPE}
 import org.apache.kyuubi.operation.{KyuubiOperation, OperationHandle}
-import org.apache.kyuubi.session.KyuubiSession
+import org.apache.kyuubi.session.{KyuubiSession, KyuubiSessionImpl}
 
 /**
  * A [[KyuubiOperationEvent]] used to tracker the lifecycle of an operation at server side.
@@ -48,6 +49,7 @@ case class KyuubiOperationEvent private (
     remoteId: String,
     statement: String,
     shouldRunAsync: Boolean,
+    operationType: String,
     state: String,
     eventTime: Long,
     createTime: Long,
@@ -56,7 +58,10 @@ case class KyuubiOperationEvent private (
     exception: Option[Throwable],
     sessionId: String,
     sessionUser: String,
-    sessionType: String) extends KyuubiEvent {
+    sessionType: String,
+    engineId: String,
+    engineType: String,
+    engineShareLevel: String) extends KyuubiEvent {
 
   // operation events are partitioned by the date when the corresponding operations are
   // created.
@@ -77,6 +82,7 @@ object KyuubiOperationEvent {
       Option(operation.remoteOpHandle()).map(OperationHandle(_).identifier.toString).orNull,
       operation.statement,
       operation.shouldRunAsync,
+      Option(operation.remoteOpHandle()).map(_.getOperationType.name()).orNull,
       status.state.name(),
       status.lastModified,
       status.create,
@@ -85,6 +91,9 @@ object KyuubiOperationEvent {
       status.exception,
       session.handle.identifier.toString,
       session.user,
-      session.sessionType.toString)
+      session.sessionType.toString,
+      session.asInstanceOf[KyuubiSessionImpl].client.engineId.orNull,
+      Option(session.sessionManager.getConf).map(_.get(ENGINE_TYPE)).orNull,
+      Option(session.sessionManager.getConf).map(_.get(ENGINE_SHARE_LEVEL)).orNull)
   }
 }
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiSessionEvent.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiSessionEvent.scala
index 821017b29..499cfc756 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiSessionEvent.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/KyuubiSessionEvent.scala
@@ -37,7 +37,10 @@ import org.apache.kyuubi.session.KyuubiSession
  * @param openedTime session opened time
  * @param endTime session end time
  * @param totalOperations how many queries and meta calls
+ * @param finishedOperations how many queries and meta calls
  * @param exception the session exception, such as the exception that occur when opening session
+ * @param runningOperations how many queries running
+ * @param errorOperations how many queries failed
  */
 case class KyuubiSessionEvent(
     sessionId: String,
@@ -55,7 +58,10 @@ case class KyuubiSessionEvent(
     var openedTime: Long = -1L,
     var endTime: Long = -1L,
     var totalOperations: Int = 0,
-    var exception: Option[Throwable] = None) extends KyuubiEvent {
+    var finishedOperations: Int = 0,
+    var exception: Option[Throwable] = None,
+    var runningOperations: Int = 0,
+    var errorOperations: Int = 0) extends KyuubiEvent {
   override def partitions: Seq[(String, String)] =
     ("day", Utils.getDateFromTimestamp(startTime)) :: Nil
 }
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
index 7019d8a6a..b299d6f17 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiRestFrontendService.scala
@@ -26,7 +26,7 @@ import javax.ws.rs.core.Response.Status
 
 import com.google.common.annotations.VisibleForTesting
 import org.apache.hadoop.conf.Configuration
-import org.eclipse.jetty.servlet.{ErrorPageErrorHandler, FilterHolder}
+import org.eclipse.jetty.servlet.{ErrorPageErrorHandler, FilterHolder, ServletContextHandler}
 
 import org.apache.kyuubi.{KyuubiException, Utils}
 import org.apache.kyuubi.config.KyuubiConf
@@ -81,14 +81,21 @@ class KyuubiRestFrontendService(override val serverable: Serverable)
   }
 
   private def startInternal(): Unit = {
-    val contextHandler = ApiRootResource.getServletHandler(this)
+    // val contextHandler = ApiRootResource.getServletHandler(this)
+    val contextHandlerCollection = ApiRootResource.getServletHandlerCollection(this)
     val holder = new FilterHolder(new AuthenticationFilter(conf))
-    contextHandler.addFilter(holder, "/v1/*", EnumSet.allOf(classOf[DispatcherType]))
+    contextHandlerCollection.getChildHandlers.map(handler => {
+      if (handler.isInstanceOf[ServletContextHandler]) {
+        handler.asInstanceOf[ServletContextHandler]
+          .addFilter(holder, "/v1/*", EnumSet.allOf(classOf[DispatcherType]))
+      }
+    })
+    // contextHandler.addFilter(holder, "/v1/*", EnumSet.allOf(classOf[DispatcherType]))
     val authenticationFactory = new KyuubiHttpAuthenticationFactory(conf)
-    server.addHandler(authenticationFactory.httpHandlerWrapperFactory.wrapHandler(contextHandler))
-
+    server.addHandler(
+      authenticationFactory.httpHandlerWrapperFactory.wrapHandler(contextHandlerCollection))
     server.addStaticHandler("org/apache/kyuubi/ui/static", "/static/")
-    server.addRedirectHandler("/", "/static/")
+    server.addRedirectHandler("/", "/static")
     server.addRedirectHandler("/static", "/static/")
     server.addStaticHandler("META-INF/resources/webjars/swagger-ui/4.9.1/", "/swagger-static/")
     server.addStaticHandler("org/apache/kyuubi/ui/swagger", "/swagger/")
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
index e81240a96..e25f4ddf2 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
@@ -17,6 +17,8 @@
 
 package org.apache.kyuubi.server
 
+import java.util
+
 import scala.util.Properties
 
 import org.apache.commons.lang3.StringUtils
@@ -31,6 +33,7 @@ import org.apache.kyuubi.events.{EventBus, KyuubiServerInfoEvent, ServerEventHan
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{AuthTypes, ServiceDiscovery}
 import org.apache.kyuubi.metrics.{MetricsConf, MetricsSystem}
+import org.apache.kyuubi.server.log.ServerLog
 import org.apache.kyuubi.server.metadata.jdbc.JDBCMetadataStoreConf
 import org.apache.kyuubi.service.{AbstractBackendService, AbstractFrontendService, Serverable, ServiceState}
 import org.apache.kyuubi.session.KyuubiSessionManager
@@ -137,6 +140,13 @@ object KyuubiServer extends Logging {
     val refreshedUnlimitedUsers = sessionMgr.getUnlimitedUsers()
     info(s"Refreshed unlimited users from $existingUnlimitedUsers to $refreshedUnlimitedUsers")
   }
+
+  private[kyuubi] def getServerLogRowSet(maxRows: Int): util.ArrayList[String] = {
+    val serverLog = ServerLog.getCurrentServerLog
+    Option(serverLog).map(_.read(maxRows)).getOrElse {
+      throw KyuubiSQLException("failed to generate server log")
+    }
+  }
 }
 
 class KyuubiServer(name: String) extends Serverable(name) {
@@ -165,6 +175,8 @@ class KyuubiServer(name: String) extends Serverable(name) {
 
   override def initialize(conf: KyuubiConf): Unit = synchronized {
     initLoggerEventHandler(conf)
+    val serverLog = ServerLog.createServerLog(conf)
+    ServerLog.setCurrentServerLog(serverLog)
 
     val kinit = new KinitAuxiliaryService()
     addService(kinit)
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTBinaryFrontendService.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTBinaryFrontendService.scala
index 069bc63e2..72d365671 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTBinaryFrontendService.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiTBinaryFrontendService.scala
@@ -39,6 +39,10 @@ final class KyuubiTBinaryFrontendService(
     override val serverable: Serverable)
   extends TBinaryFrontendService("KyuubiTBinaryFrontend") {
 
+  override def attributes: Map[String, String] = {
+    Map(KYUUBI_SERVER_SUBMIT_TIME -> System.currentTimeMillis().toString)
+  }
+
   override protected def hadoopConf: Configuration = KyuubiServer.getHadoopConf()
 
   override lazy val discoveryService: Option[Service] = {
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/WebUIProxyServlet.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/WebUIProxyServlet.scala
new file mode 100644
index 000000000..d0d530c78
--- /dev/null
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/WebUIProxyServlet.scala
@@ -0,0 +1,81 @@
+/*
+ * 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.api
+
+import java.net.URLDecoder
+import javax.servlet.http.HttpServletRequest
+
+import org.eclipse.jetty.client.api.Request
+import org.eclipse.jetty.proxy.ProxyServlet
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.config.KyuubiConf
+
+private[api] class WebUIProxyServlet(conf: KyuubiConf) extends ProxyServlet with Logging {
+  var ipAddress: String = _
+  var port: Int = _
+  val CONTEXT_HEADER_KEY = "X-Forwarded-Context"
+  var CONTEXT_HEADER_VALUE = ""
+
+  override def rewriteTarget(request: HttpServletRequest): String = {
+    var targetUrl = "/no-ui-error"
+    val requestUrl = request.getRequestURI
+    logger.info("requestUrl is {}", requestUrl)
+    val url = requestUrl.split("/")
+    logger.info("url is {}", url)
+    if (url != null && url.length > 0) {
+      ipAddress = url(2).split(":")(0)
+      port = url(2).split(":")(1).toInt
+      var targetPath = "jobs/"
+      val subPath = String.format(
+        "/%s/%s/",
+        url(1),
+        url(2))
+      if (requestUrl.substring(subPath.length).nonEmpty) {
+        targetPath = requestUrl.substring(subPath.length)
+      }
+      targetUrl = String.format(
+        "http://%s:%s/%s%s",
+        ipAddress,
+        port.toString,
+        targetPath,
+        getQueryString(request))
+      CONTEXT_HEADER_VALUE = s"/proxy/$ipAddress:$port"
+      logger.info("ui -> {}", targetUrl)
+    }
+    targetUrl
+  }
+
+  override def addXForwardedHeaders(
+      clientRequest: HttpServletRequest,
+      proxyRequest: Request): Unit = {
+    proxyRequest.header(CONTEXT_HEADER_KEY, CONTEXT_HEADER_VALUE)
+    super.addXForwardedHeaders(clientRequest, proxyRequest)
+  }
+
+  def getQueryString(servletRequest: HttpServletRequest): String = {
+    val result = new StringBuilder()
+    val queryString = servletRequest.getQueryString()
+    if (queryString != null && queryString.length() > 0) {
+      result.append('?')
+      result.append(URLDecoder.decode(queryString, "UTF-8"))
+    }
+    result.toString()
+  }
+
+}
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 104dd1045..1533d8d54 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
@@ -23,20 +23,25 @@ import javax.ws.rs.core.{MediaType, Response}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ListBuffer
+import scala.util.control.NonFatal
 
-import io.swagger.v3.oas.annotations.media.Content
+import io.swagger.v3.oas.annotations.media.{Content, Schema}
 import io.swagger.v3.oas.annotations.responses.ApiResponse
 import io.swagger.v3.oas.annotations.tags.Tag
+import org.apache.commons.lang3.StringUtils
 
 import org.apache.kyuubi.{KYUUBI_VERSION, Logging, Utils}
-import org.apache.kyuubi.client.api.v1.dto.Engine
+import org.apache.kyuubi.client.api.v1.dto.{Engine, HadoopConfData, Server, ServerLog}
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.config.KyuubiReservedKeys._
 import org.apache.kyuubi.ha.HighAvailabilityConf.HA_NAMESPACE
 import org.apache.kyuubi.ha.client.{DiscoveryPaths, ServiceNodeInfo}
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider.withDiscoveryClient
 import org.apache.kyuubi.server.KyuubiServer
+import org.apache.kyuubi.server.KyuubiServer.getServerLogRowSet
 import org.apache.kyuubi.server.api.ApiRequestContext
+import org.apache.kyuubi.util.OSUtils
 
 @Tag(name = "Admin")
 @Produces(Array(MediaType.APPLICATION_JSON))
@@ -109,14 +114,13 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
   @DELETE
   @Path("engine")
   def deleteEngine(
-      @QueryParam("type") engineType: String,
-      @QueryParam("sharelevel") shareLevel: String,
+      @QueryParam("engineType") engineType: String,
+      @QueryParam("shareLevel") shareLevel: String,
       @QueryParam("subdomain") subdomain: String,
       @QueryParam("hive.server2.proxy.user") hs2ProxyUser: String): Response = {
     val userName = fe.getSessionUser(hs2ProxyUser)
     val engine = getEngine(userName, engineType, shareLevel, subdomain, "default")
     val engineSpace = getEngineSpace(engine)
-
     withDiscoveryClient(fe.getConf) { discoveryClient =>
       val engineNodes = discoveryClient.getChildren(engineSpace)
       engineNodes.foreach { node =>
@@ -138,44 +142,152 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
 
   @ApiResponse(
     responseCode = "200",
-    content = Array(new Content(mediaType = MediaType.APPLICATION_JSON)),
-    description = "list kyuubi engines")
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[Engine]))),
+    description = "list alive kyuubi engines")
   @GET
   @Path("engine")
   def listEngines(
-      @QueryParam("type") engineType: String,
-      @QueryParam("sharelevel") shareLevel: String,
+      @QueryParam("engineType") engineType: String,
+      @QueryParam("shareLevel") shareLevel: String,
       @QueryParam("subdomain") subdomain: String,
       @QueryParam("hive.server2.proxy.user") hs2ProxyUser: String): Seq[Engine] = {
-    val userName = fe.getSessionUser(hs2ProxyUser)
-    val engine = getEngine(userName, engineType, shareLevel, subdomain, "")
-    val engineSpace = getEngineSpace(engine)
-
-    var engineNodes = ListBuffer[ServiceNodeInfo]()
-    Option(subdomain).filter(_.nonEmpty) match {
-      case Some(_) =>
-        withDiscoveryClient(fe.getConf) { discoveryClient =>
-          info(s"Listing engine nodes for $engineSpace")
-          engineNodes ++= discoveryClient.getServiceNodesInfo(engineSpace)
-        }
-      case None =>
-        withDiscoveryClient(fe.getConf) { discoveryClient =>
-          discoveryClient.getChildren(engineSpace).map { child =>
-            info(s"Listing engine nodes for $engineSpace/$child")
-            engineNodes ++= discoveryClient.getServiceNodesInfo(s"$engineSpace/$child")
+    val engines = ListBuffer[Engine]()
+    try {
+      val userName = if (StringUtils.isEmpty(hs2ProxyUser)) "" else fe.getSessionUser(hs2ProxyUser)
+      val engine = getEngine(userName, engineType, shareLevel, subdomain, "")
+      val engineSpace = getEngineSpace(engine)
+      val engineNodes = ListBuffer[ServiceNodeInfo]()
+      Option(subdomain).filter(_.nonEmpty) match {
+        case Some(_) =>
+          withDiscoveryClient(fe.getConf) { discoveryClient =>
+            info(s"Listing engine nodes for $engineSpace")
+            engineNodes ++= discoveryClient.getServiceNodesInfo(engineSpace)
+          }
+        case None =>
+          withDiscoveryClient(fe.getConf) { discoveryClient =>
+            discoveryClient.getChildren(engineSpace).map { child =>
+              {
+                if (StringUtils.isNotEmpty(hs2ProxyUser)) {
+                  info(s"Listing engine nodes for $engineSpace/$child")
+                  engineNodes ++= discoveryClient.getServiceNodesInfo(s"$engineSpace/$child")
+                } else {
+                  discoveryClient.getChildren(s"$engineSpace/$child").map(restChild => {
+                    info(s"Listing engine nodes for $engineSpace/$child/$restChild")
+                    engineNodes ++= discoveryClient
+                      .getServiceNodesInfo(s"$engineSpace/$child/$restChild")
+                  })
+                }
+              }
+            }
           }
+      }
+      engineNodes.map(node => {
+        engines += new Engine(
+          engine.getVersion,
+          engine.getUser,
+          engine.getEngineType,
+          engine.getSharelevel,
+          node.namespace.split("/").last,
+          node.instance,
+          node.namespace,
+          node.attributes.asJava,
+          node.attributes.get(KYUUBI_ENGINE_SUBMIT_TIME_KEY).orNull,
+          node.attributes.get(KYUUBI_ENGINE_URL).orNull,
+          node.host,
+          node.port,
+          node.engineRefId.orNull,
+          "Running",
+          node.attributes.get(KYUUBI_ENGINE_MEMORY).orNull,
+          node.attributes.get(KYUUBI_ENGINE_CPU).orNull)
+      })
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = s"Error getting engines"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+    engines
+  }
+
+  @deprecated
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(mediaType = MediaType.APPLICATION_JSON)),
+    description = "list  all live kyuubi servers")
+  @GET
+  @Path("servers")
+  def listServers(@QueryParam("host") @DefaultValue("") host: String): Seq[Server] = {
+    val Servers = ListBuffer[Server]()
+    val serverSpace = DiscoveryPaths.makePath(null, fe.getConf.get(HA_NAMESPACE))
+    val serverNodes = ListBuffer[ServiceNodeInfo]()
+    withDiscoveryClient(fe.getConf) { discoveryClient =>
+      info(s"Listing server nodes for $serverSpace")
+      serverNodes ++= discoveryClient.getServiceNodesInfo(serverSpace)
+      serverNodes.map(node => {
+        if (host.equalsIgnoreCase("") || node.host.equalsIgnoreCase(host)) {
+          Servers += new Server(
+            node.nodeName,
+            node.namespace,
+            node.instance,
+            node.host,
+            node.port,
+            node.attributes.get(KYUUBI_SERVER_SUBMIT_TIME).orNull.toLong,
+            OSUtils.memoryTotal(),
+            OSUtils.cpuTotal(),
+            "Running")
         }
+      })
     }
-    engineNodes.map(node =>
-      new Engine(
-        engine.getVersion,
-        engine.getUser,
-        engine.getEngineType,
-        engine.getSharelevel,
-        node.namespace.split("/").last,
-        node.instance,
-        node.namespace,
-        node.attributes.asJava))
+    Servers
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[ServerLog]))),
+    description =
+      "get server log")
+  @GET
+  @Path("server/log")
+  def getServerLog(
+      @QueryParam("maxrows") maxRows: Int): ServerLog = {
+    try {
+      val rowSet = getServerLogRowSet(maxRows)
+      new ServerLog(rowSet, rowSet.size)
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = s"Error getting server log"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(mediaType = MediaType.APPLICATION_JSON)),
+    description = "get the Kyuubi server hadoop conf")
+  @GET
+  @Path("get/hadoop_conf")
+  def getFrontendHadoopConf(): Seq[HadoopConfData] = {
+    val userName = fe.getSessionUser(Map.empty[String, String])
+    val ipAddress = fe.getIpAddress
+    info(s"Receive get Kyuubi server hadoop conf request from $userName/$ipAddress")
+    info(s"the admin is $administrator")
+    if (!userName.equals(administrator)) {
+      throw new NotAllowedException(
+        s"$userName is not allowed to get the Kyuubi server hadoop conf")
+    }
+    info(s"Getting the Kyuubi server hadoop conf")
+    val hadoopConf = ListBuffer[HadoopConfData]()
+    val iterator = KyuubiServer.getHadoopConf().iterator()
+    while (iterator.hasNext()) {
+      val element = iterator.next();
+      hadoopConf += new HadoopConfData(element.getKey, element.getValue)
+    }
+    hadoopConf
   }
 
   private def getEngine(
@@ -203,7 +315,15 @@ private[v1] class AdminResource extends ApiRequestContext with Logging {
       engineSubdomain,
       null,
       null,
-      Collections.emptyMap())
+      Collections.emptyMap(),
+      null,
+      null,
+      null,
+      0,
+      null,
+      null,
+      null,
+      null)
   }
 
   private def getEngineSpace(engine: Engine): String = {
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala
index d8b997e86..1dcdc8d4a 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/ApiRootResource.scala
@@ -23,14 +23,16 @@ import javax.ws.rs.core.{MediaType, Response}
 import com.google.common.annotations.VisibleForTesting
 import io.swagger.v3.oas.annotations.media.Content
 import io.swagger.v3.oas.annotations.responses.ApiResponse
+import org.eclipse.jetty.server.handler.ContextHandlerCollection
 import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder}
 import org.glassfish.jersey.server.ResourceConfig
 import org.glassfish.jersey.servlet.ServletContainer
 
 import org.apache.kyuubi.KYUUBI_VERSION
 import org.apache.kyuubi.client.api.v1.dto._
+import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.server.KyuubiRestFrontendService
-import org.apache.kyuubi.server.api.{ApiRequestContext, FrontendServiceContext, OpenAPIConfig}
+import org.apache.kyuubi.server.api._
 
 @Path("/v1")
 private[v1] class ApiRootResource extends ApiRequestContext {
@@ -82,4 +84,28 @@ private[server] object ApiRootResource {
     handler.addServlet(holder, "/*")
     handler
   }
+
+  def getServletHandlerCollection(fe: KyuubiRestFrontendService): ContextHandlerCollection = {
+    val contextHandlerCollection = new ContextHandlerCollection();
+    val openapiConf: ResourceConfig = new OpenAPIConfig
+    val holder = new ServletHolder(new ServletContainer(openapiConf))
+    val handler = new ServletContextHandler(ServletContextHandler.NO_SESSIONS)
+    handler.setContextPath("/api")
+    FrontendServiceContext.set(handler, fe)
+    handler.addServlet(holder, "/*")
+    contextHandlerCollection.addHandler(handler)
+    addProxyHandler(contextHandlerCollection, fe.getConf)
+    contextHandlerCollection
+  }
+
+  def addProxyHandler(
+      contextHandlerCollection: ContextHandlerCollection,
+      conf: KyuubiConf): Unit = {
+    val proxyServlet = new WebUIProxyServlet(conf)
+    val holder = new ServletHolder(proxyServlet)
+    val contextHandler = new ServletContextHandler(ServletContextHandler.NO_SESSIONS);
+    contextHandler.setContextPath("/proxy");
+    contextHandler.addServlet(holder, "/*")
+    contextHandlerCollection.addHandler(contextHandler)
+  }
 }
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/OperationsResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/OperationsResource.scala
index 70a6d3a28..f67964ab0 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/OperationsResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/OperationsResource.scala
@@ -21,6 +21,7 @@ import javax.ws.rs._
 import javax.ws.rs.core.{MediaType, Response}
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
 import scala.util.control.NonFatal
 
 import io.swagger.v3.oas.annotations.media.{Content, Schema}
@@ -31,7 +32,7 @@ import org.apache.hive.service.rpc.thrift._
 import org.apache.kyuubi.{KyuubiSQLException, Logging}
 import org.apache.kyuubi.client.api.v1.dto._
 import org.apache.kyuubi.events.KyuubiOperationEvent
-import org.apache.kyuubi.operation.{FetchOrientation, KyuubiOperation, OperationHandle}
+import org.apache.kyuubi.operation.{FetchOrientation, KyuubiOperation, OperationHandle, OperationStatus}
 import org.apache.kyuubi.server.api.ApiRequestContext
 
 @Tag(name = "Operation")
@@ -79,7 +80,7 @@ private[v1] class OperationsResource extends ApiRequestContext with Logging {
         case "close" => fe.be.closeOperation(operationHandle)
         case _ => throw KyuubiSQLException(s"Invalid action ${request.getAction}")
       }
-      Response.ok().build()
+      Response.ok("Apply an action for an operation successfully.").build()
     } catch {
       case NonFatal(e) =>
         val errorMsg =
@@ -237,4 +238,113 @@ private[v1] class OperationsResource extends ApiRequestContext with Logging {
         throw new NotFoundException(errorMsg)
     }
   }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[InfoDetail]))),
+    description =
+      "all the enum values of OperationType")
+  @GET
+  @Path("/operations/types")
+  def getSupportedInfoType(): Seq[InfoDetail] = {
+    try {
+      val infoTypes = TOperationType.values()
+      infoTypes.map(infoType => {
+        new InfoDetail(infoType.toString, infoType.getValue.toString)
+      })
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = "Error getting all the enum values of OperationType"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[OperationStatus]))),
+    description =
+      "Get the operation status")
+  @POST
+  @Path("{operationHandle}/status")
+  def getOperationStatus(
+      @PathParam("operationHandle") operationHandleStr: String): OperationStatus = {
+    try {
+      val opHandle = OperationHandle(operationHandleStr)
+      val operation = fe.be.sessionManager.operationManager.getOperation(opHandle)
+      operation.getStatus
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = "Error getting an operation status"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[String]))),
+    description =
+      "Get the query id of the given operation identifier")
+  @GET
+  @Path("{operationHandle}/queryId")
+  def getQueryId(
+      @PathParam("operationHandle") operationHandleStr: String): String = {
+    try {
+      val opHandle = OperationHandle(operationHandleStr)
+      val operation = fe.be.sessionManager.operationManager.getOperation(opHandle)
+      val queryId = fe.be.sessionManager.operationManager.getQueryId(operation)
+      queryId
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = "Error getting an the query id of operation"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[KyuubiOperationEvent]))),
+    description =
+      "query operation event list")
+  @GET
+  @Path("listOperation")
+  def listOperation(
+      @QueryParam("sessionHandle") @DefaultValue("") sessionHandleStr: String,
+      @QueryParam("operationType") @DefaultValue("") operationType: String,
+      @QueryParam("state") @DefaultValue("") stateStr: String): Seq[KyuubiOperationEvent] = {
+    try {
+      val KyuubiOperationEvents = ListBuffer[KyuubiOperationEvent]()
+      fe.be.sessionManager.allSessions()
+        .map { session =>
+          session.allOperations().map { operationHandle =>
+            val operation = fe.be.sessionManager.operationManager.getOperation(operationHandle)
+            val kyuubiOperationEvent = KyuubiOperationEvent(operation.asInstanceOf[KyuubiOperation])
+            KyuubiOperationEvents += kyuubiOperationEvent
+          }
+        }
+      (KyuubiOperationEvents
+        .filter(
+          sessionHandleStr.equalsIgnoreCase("") || _.sessionId.equalsIgnoreCase(sessionHandleStr))
+        .filter(
+          operationType.equalsIgnoreCase("") || _.operationType.equalsIgnoreCase(operationType))
+        .filter(
+          stateStr.equalsIgnoreCase("") || _.state.equalsIgnoreCase(stateStr)))
+
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = "Error querying operation event list"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
 }
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
index 84b19eb00..7ac2d53dc 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/api/v1/SessionsResource.scala
@@ -21,6 +21,7 @@ import javax.ws.rs._
 import javax.ws.rs.core.{MediaType, Response}
 
 import scala.collection.JavaConverters._
+import scala.collection.mutable.ListBuffer
 import scala.language.implicitConversions
 import scala.util.control.NonFatal
 
@@ -33,11 +34,10 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.client.api.v1.dto
 import org.apache.kyuubi.client.api.v1.dto._
 import org.apache.kyuubi.config.KyuubiReservedKeys._
-import org.apache.kyuubi.events.KyuubiEvent
-import org.apache.kyuubi.operation.OperationHandle
+import org.apache.kyuubi.events.{KyuubiEvent, KyuubiOperationEvent, KyuubiSessionEvent}
+import org.apache.kyuubi.operation.{KyuubiOperation, OperationHandle}
 import org.apache.kyuubi.server.api.ApiRequestContext
-import org.apache.kyuubi.session.KyuubiSession
-import org.apache.kyuubi.session.SessionHandle
+import org.apache.kyuubi.session.{KyuubiSession, SessionHandle}
 
 @Tag(name = "Session")
 @Produces(Array(MediaType.APPLICATION_JSON))
@@ -85,6 +85,34 @@ private[v1] class SessionsResource extends ApiRequestContext with Logging {
     }
   }
 
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[KyuubiEvent]))),
+    description = "get session info")
+  @GET
+  @Path("listSessionInfo")
+  def listSessionInfo(
+      @QueryParam("user") @DefaultValue("") user: String,
+      @QueryParam("serverIP") @DefaultValue("") serverIP: String): Seq[KyuubiSessionEvent] = {
+    try {
+      val kyuubiSessionEvents = ListBuffer[KyuubiSessionEvent]()
+      sessionManager.allSessions().map { session =>
+        kyuubiSessionEvents += sessionManager.getSession(session.handle.identifier.toString)
+          .asInstanceOf[KyuubiSession].getSessionEvent.get
+      }
+      (kyuubiSessionEvents
+        .filter(serverIP.equalsIgnoreCase("") || _.serverIP.equalsIgnoreCase(serverIP))
+        .filter(user.equalsIgnoreCase("") || _.user.equalsIgnoreCase(user)))
+    } catch {
+      case NonFatal(e) =>
+        val errorMsg = "Error getting all session info"
+        error(errorMsg, e)
+        throw new NotFoundException(errorMsg)
+    }
+  }
+
   @ApiResponse(
     responseCode = "200",
     content = Array(new Content(
@@ -404,4 +432,87 @@ private[v1] class SessionsResource extends ApiRequestContext with Logging {
         throw new NotFoundException(errorMsg)
     }
   }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[SQLDetail]))),
+    description =
+      "get sql detail list hosted a specific session binding via an identifier")
+  @GET
+  @Path("{sessionHandle}/sqlDetails")
+  def getOperations(
+      @PathParam("sessionHandle") sessionHandleStr: String): Seq[SQLDetail] = {
+    try {
+      sessionManager.getSession(sessionHandleStr)
+        .allOperations().map { operationHandle =>
+          val operation = fe.be.sessionManager.operationManager.getOperation(operationHandle)
+          val kyuubiSessionEvent = KyuubiOperationEvent(operation.asInstanceOf[KyuubiOperation])
+          new SQLDetail(
+            sessionHandleStr,
+            kyuubiSessionEvent.sessionUser,
+            kyuubiSessionEvent.statementId,
+            kyuubiSessionEvent.createTime,
+            kyuubiSessionEvent.completeTime,
+            kyuubiSessionEvent.statement,
+            kyuubiSessionEvent.engineId,
+            kyuubiSessionEvent.engineType,
+            kyuubiSessionEvent.engineShareLevel,
+            kyuubiSessionEvent.exception.map(_.toString).orNull)
+        }.toSeq
+    } catch {
+      case NonFatal(e) =>
+        error(s"Invalid $sessionHandleStr", e)
+        throw new NotFoundException(s"Invalid $sessionHandleStr")
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[InfoDetail]))),
+    description =
+      "get all supported info types by Kyuubi session")
+  @GET
+  @Path("{sessionHandle}/infoTypes")
+  def getSupportedInfoType(
+      @PathParam("sessionHandle") sessionHandleStr: String): Seq[InfoDetail] = {
+    try {
+      val infoTypes = TGetInfoType.values()
+      infoTypes.map(infoType => {
+        new InfoDetail(infoType.toString, infoType.getValue.toString)
+      }).toSeq
+    } catch {
+      case NonFatal(e) =>
+        error(s"Invalid $sessionHandleStr", e)
+        throw new NotFoundException(s"Invalid $sessionHandleStr")
+    }
+  }
+
+  @ApiResponse(
+    responseCode = "200",
+    content = Array(new Content(
+      mediaType = MediaType.APPLICATION_JSON,
+      schema = new Schema(implementation = classOf[KyuubiOperationEvent]))),
+    description =
+      "get all the operation event hosted a specific session binding via an identifier")
+  @GET
+  @Path("{sessionHandle}/operations")
+  def getAllOperationEvent(
+      @PathParam("sessionHandle") sessionHandleStr: String): Seq[KyuubiOperationEvent] = {
+    try {
+      sessionManager.getSession(sessionHandleStr)
+        .allOperations().map { operationHandle =>
+          val operation = fe.be.sessionManager.operationManager.getOperation(operationHandle)
+          KyuubiOperationEvent(operation.asInstanceOf[KyuubiOperation])
+        }.toSeq
+    } catch {
+      case NonFatal(e) =>
+        error(s"Invalid $sessionHandleStr", e)
+        throw new NotFoundException(s"Invalid $sessionHandleStr")
+    }
+  }
+
 }
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
index e4203b301..93381cd1c 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSessionImpl.scala
@@ -206,9 +206,19 @@ class KyuubiSessionImpl(
           operation.close()
           throw t
       }
-      sessionEvent.totalOperations += 1
     }
-    super.runOperation(operation)
+    try {
+      sessionEvent.runningOperations += 1
+      val OperationHandle = super.runOperation(operation)
+      sessionEvent.runningOperations -= 1
+      sessionEvent.finishedOperations += 1
+      OperationHandle
+    } catch {
+      case e: Exception =>
+        sessionEvent.errorOperations += 1
+        throw e
+    }
+
   }
 
   @volatile private var engineLaunched: Boolean = false
diff --git a/kyuubi-server/web-ui/src/router/contact/index.ts b/kyuubi-server/src/main/scala/org/apache/kyuubi/util/OSUtils.scala
similarity index 68%
copy from kyuubi-server/web-ui/src/router/contact/index.ts
copy to kyuubi-server/src/main/scala/org/apache/kyuubi/util/OSUtils.scala
index a83c653ec..5754747bd 100644
--- a/kyuubi-server/web-ui/src/router/contact/index.ts
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/util/OSUtils.scala
@@ -15,12 +15,23 @@
  * limitations under the License.
  */
 
-const routes = [
-  {
-    path: '/contact',
-    name: 'contact',
-    component: () => import('@/views/contact/index.vue')
+package org.apache.kyuubi.util
+
+import oshi.SystemInfo
+
+object OSUtils {
+  private val SI = new SystemInfo
+
+  private val hal = SI.getHardware
+
+  def memoryTotal(): Long = {
+    val memory = hal.getMemory()
+    val memoryTotal = memory.getTotal
+    memoryTotal
   }
-]
 
-export default routes
+  def cpuTotal(): Int = {
+    val processor = hal.getProcessor()
+    processor.getLogicalProcessorCount
+  }
+}
diff --git a/kyuubi-server/web-ui/package.json b/kyuubi-server/web-ui/package.json
index 63fdc7221..131e69b7f 100644
--- a/kyuubi-server/web-ui/package.json
+++ b/kyuubi-server/web-ui/package.json
@@ -17,6 +17,7 @@
   "dependencies": {
     "@element-plus/icons-vue": "^2.0.9",
     "axios": "^0.27.2",
+    "date-fns": "^2.29.3",
     "element-plus": "^2.2.12",
     "pinia": "^2.0.18",
     "pinia-plugin-persistedstate": "^2.1.1",
diff --git a/kyuubi-server/web-ui/pnpm-lock.yaml b/kyuubi-server/web-ui/pnpm-lock.yaml
index 61fc5124d..c44c785bc 100644
--- a/kyuubi-server/web-ui/pnpm-lock.yaml
+++ b/kyuubi-server/web-ui/pnpm-lock.yaml
@@ -12,6 +12,7 @@ specifiers:
   '@vue/eslint-config-typescript': ^11.0.0
   '@vue/test-utils': ^2.0.2
   axios: ^0.27.2
+  date-fns: ^2.29.3
   element-plus: ^2.2.12
   eslint: ^8.21.0
   eslint-plugin-prettier: ^4.2.1
@@ -30,80 +31,90 @@ specifiers:
   vue-tsc: ^0.38.4
 
 dependencies:
-  '@element-plus/icons-vue': 2.0.9_vue@3.2.37
+  '@element-plus/icons-vue': 2.0.10_vue@3.2.45
   axios: 0.27.2
-  element-plus: 2.2.13_vue@3.2.37
-  pinia: 2.0.18_j6bzmzd4ujpabbp5objtwxyjp4
-  pinia-plugin-persistedstate: 2.1.1_pinia@2.0.18
-  vue: 3.2.37
-  vue-i18n: 9.2.2_vue@3.2.37
-  vue-router: 4.1.3_vue@3.2.37
+  date-fns: 2.29.3
+  element-plus: 2.2.26_vue@3.2.45
+  pinia: 2.0.27_mgnvym7yiazkylwwogi5r767ue
+  pinia-plugin-persistedstate: 2.4.0_pinia@2.0.27
+  vue: 3.2.45
+  vue-i18n: 9.2.2_vue@3.2.45
+  vue-router: 4.1.6_vue@3.2.45
 
 devDependencies:
-  '@iconify-json/ep': 1.1.7
-  '@types/node': 18.7.6
-  '@typescript-eslint/eslint-plugin': 5.33.1_vsoshirnpb7xw6mr7xomgfas2i
-  '@typescript-eslint/parser': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
-  '@vitejs/plugin-vue': 3.0.3_vite@3.0.8+vue@3.2.37
-  '@vitest/coverage-c8': 0.22.0_jsdom@20.0.0+sass@1.54.4
-  '@vue/eslint-config-prettier': 7.0.0_2xd4q2tc5cqa5as7uugqhp6oue
-  '@vue/eslint-config-typescript': 11.0.0_4py5zxx5ck6utobkmozwvrmyiy
-  '@vue/test-utils': 2.0.2_vue@3.2.37
-  eslint: 8.22.0
-  eslint-plugin-prettier: 4.2.1_2xd4q2tc5cqa5as7uugqhp6oue
-  eslint-plugin-vue: 9.3.0_eslint@8.22.0
-  jsdom: 20.0.0
-  prettier: 2.7.1
-  sass: 1.54.4
-  typescript: 4.7.4
-  vite: 3.0.8_sass@1.54.4
-  vitest: 0.22.0_jsdom@20.0.0+sass@1.54.4
-  vue-tsc: 0.38.9_typescript@4.7.4
+  '@iconify-json/ep': 1.1.8
+  '@types/node': 18.11.11
+  '@typescript-eslint/eslint-plugin': 5.45.1_tdm6ms4ntwhlpozn7kjqrhum74
+  '@typescript-eslint/parser': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
+  '@vitejs/plugin-vue': 3.2.0_vite@3.2.5+vue@3.2.45
+  '@vitest/coverage-c8': 0.22.1_jsdom@20.0.3+sass@1.56.1
+  '@vue/eslint-config-prettier': 7.0.0_yz7jvh5l5mgl5xhpwlgtkgcccy
+  '@vue/eslint-config-typescript': 11.0.2_sb3redb42nwkq4q6vnqovjecl4
+  '@vue/test-utils': 2.2.6_vue@3.2.45
+  eslint: 8.29.0
+  eslint-plugin-prettier: 4.2.1_yz7jvh5l5mgl5xhpwlgtkgcccy
+  eslint-plugin-vue: 9.8.0_eslint@8.29.0
+  jsdom: 20.0.3
+  prettier: 2.8.0
+  sass: 1.56.1
+  typescript: 4.9.3
+  vite: 3.2.5_tr7neyh64qpc4cjqb6wcsya4ri
+  vitest: 0.22.1_jsdom@20.0.3+sass@1.56.1
+  vue-tsc: 0.38.9_typescript@4.9.3
 
 packages:
 
-  /@babel/helper-string-parser/7.18.10:
-    resolution: {integrity: sha512-XtIfWmeNY3i4t7t4D2t02q50HvqHybPqW2ki1kosnvWCwuCMeo81Jf0gwr85jy/neUdg5XDdeFE/80DXiO+njw==}
+  /@babel/helper-string-parser/7.19.4:
+    resolution: {integrity: sha512-nHtDoQcuqFmwYNYPz3Rah5ph2p8PFeFCsZk9A/48dPc/rGocJ5J3hAAZ7pb76VWX3fZKu+uEr/FhH5jLx7umrw==}
     engines: {node: '>=6.9.0'}
 
-  /@babel/helper-validator-identifier/7.18.6:
-    resolution: {integrity: sha512-MmetCkz9ej86nJQV+sFCxoGGrUbU3q02kgLciwkrt9QqEB7cP39oKEY0PakknEO0Gu20SskMRi+AYZ3b1TpN9g==}
+  /@babel/helper-validator-identifier/7.19.1:
+    resolution: {integrity: sha512-awrNfaMtnHUr653GgGEs++LlAvW6w+DcPrOliSMXWCKo597CwL5Acf/wWdNkf/tfEQE3mjkeD1YOVZOUV/od1w==}
     engines: {node: '>=6.9.0'}
 
-  /@babel/parser/7.18.11:
-    resolution: {integrity: sha512-9JKn5vN+hDt0Hdqn1PiJ2guflwP+B6Ga8qbDuoF0PzzVhrzsKIJo8yGqVk6CmMHiMei9w1C1Bp9IMJSIK+HPIQ==}
+  /@babel/parser/7.20.5:
+    resolution: {integrity: sha512-r27t/cy/m9uKLXQNWWebeCUHgnAZq0CpG1OwKRxzJMP1vpSU4bSIK2hq+/cp0bQxetkXx38n09rNu8jVkcK/zA==}
     engines: {node: '>=6.0.0'}
     hasBin: true
     dependencies:
-      '@babel/types': 7.18.10
+      '@babel/types': 7.20.5
 
-  /@babel/types/7.18.10:
-    resolution: {integrity: sha512-MJvnbEiiNkpjo+LknnmRrqbY1GPUUggjv+wQVjetM/AONoupqRALB7I6jGqNUAZsKcRIEu2J6FRFvsczljjsaQ==}
+  /@babel/types/7.20.5:
+    resolution: {integrity: sha512-c9fst/h2/dcF7H+MJKZ2T0KjEQ8hY/BNnDk/H3XY8C4Aw/eWQXWn/lWntHF9ooUBnGmEvbfGrTgLWc+um0YDUg==}
     engines: {node: '>=6.9.0'}
     dependencies:
-      '@babel/helper-string-parser': 7.18.10
-      '@babel/helper-validator-identifier': 7.18.6
+      '@babel/helper-string-parser': 7.19.4
+      '@babel/helper-validator-identifier': 7.19.1
       to-fast-properties: 2.0.0
 
   /@bcoe/v8-coverage/0.2.3:
     resolution: {integrity: sha512-0hYQ8SB4Db5zvZB4axdMHGwEaQjkZzFjQiN9LVYvIFB2nSUHW9tYpxWriPrWDASIxiaXax83REcLxuSdnGPZtw==}
     dev: true
 
-  /@ctrl/tinycolor/3.4.1:
-    resolution: {integrity: sha512-ej5oVy6lykXsvieQtqZxCOaLT+xD4+QNarq78cIYISHmZXshCvROLudpQN3lfL8G0NL7plMSSK+zlyvCaIJ4Iw==}
+  /@ctrl/tinycolor/3.5.0:
+    resolution: {integrity: sha512-tlJpwF40DEQcfR/QF+wNMVyGMaO9FQp6Z1Wahj4Gk3CJQYHwA2xVG7iKDFdW6zuxZY9XWOpGcfNCTsX4McOsOg==}
     engines: {node: '>=10'}
     dev: false
 
-  /@element-plus/icons-vue/2.0.9_vue@3.2.37:
-    resolution: {integrity: sha512-okdrwiVeKBmW41Hkl0eMrXDjzJwhQMuKiBOu17rOszqM+LS/yBYpNQNV5Jvoh06Wc+89fMmb/uhzf8NZuDuUaQ==}
+  /@element-plus/icons-vue/2.0.10_vue@3.2.45:
+    resolution: {integrity: sha512-ygEZ1mwPjcPo/OulhzLE7mtDrQBWI8vZzEWSNB2W/RNCRjoQGwbaK4N8lV4rid7Ts4qvySU3njMN7YCiSlSaTQ==}
     peerDependencies:
       vue: ^3.2.0
     dependencies:
-      vue: 3.2.37
+      vue: 3.2.45
     dev: false
 
-  /@esbuild/linux-loong64/0.14.54:
-    resolution: {integrity: sha512-bZBrLAIX1kpWelV0XemxBZllyRmM6vgFQQG2GdNb+r3Fkp0FOh1NJSvekXDs7jq70k4euu1cryLMfU+mTXlEpw==}
+  /@esbuild/android-arm/0.15.18:
+    resolution: {integrity: sha512-5GT+kcs2WVGjVs7+boataCkO5Fg0y4kCjzkB5bAip7H4jfnOS3dA6KPiww9W1OEKTKeAcUVhdZGvgI65OXmUnw==}
+    engines: {node: '>=12'}
+    cpu: [arm]
+    os: [android]
+    requiresBuild: true
+    dev: true
+    optional: true
+
+  /@esbuild/linux-loong64/0.15.18:
+    resolution: {integrity: sha512-L4jVKS82XVhw2nvzLg/19ClLWg0y27ulRwuP7lcyL6AbUWB5aPglXY3M21mauDQMDfRLs8cQmeT03r/+X3cZYQ==}
     engines: {node: '>=12'}
     cpu: [loong64]
     os: [linux]
@@ -111,15 +122,15 @@ packages:
     dev: true
     optional: true
 
-  /@eslint/eslintrc/1.3.0:
-    resolution: {integrity: sha512-UWW0TMTmk2d7hLcWD1/e2g5HDM/HQ3csaLSqXCfqwh4uNDuNqlaKWXmEsL4Cs41Z0KnILNvwbHAah3C2yt06kw==}
+  /@eslint/eslintrc/1.3.3:
+    resolution: {integrity: sha512-uj3pT6Mg+3t39fvLrj8iuCIJ38zKO9FpGtJ4BBJebJhEwjoT+KLVNCcHT5QC9NGRIEi7fZ0ZR8YRb884auB4Lg==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dependencies:
       ajv: 6.12.6
       debug: 4.3.4
-      espree: 9.3.3
-      globals: 13.17.0
-      ignore: 5.2.0
+      espree: 9.4.1
+      globals: 13.18.0
+      ignore: 5.2.1
       import-fresh: 3.3.0
       js-yaml: 4.1.0
       minimatch: 3.1.2
@@ -128,18 +139,18 @@ packages:
       - supports-color
     dev: true
 
-  /@floating-ui/core/0.7.3:
-    resolution: {integrity: sha512-buc8BXHmG9l82+OQXOFU3Kr2XQx9ys01U/Q9HMIrZ300iLc8HLMgh7dcCqgYzAzf4BkoQvDcXf5Y+CuEZ5JBYg==}
+  /@floating-ui/core/1.0.3:
+    resolution: {integrity: sha512-27FDAEVHrAEQI1UV+7FIjZrFK862gBoAG0USoPMU7RoBCmaTDt6bnKVW/J2uPnOPI6TWqiWGtS7RFN+tN/k+vQ==}
     dev: false
 
-  /@floating-ui/dom/0.5.4:
-    resolution: {integrity: sha512-419BMceRLq0RrmTSDxn8hf9R3VCJv2K9PUfugh5JyEFmdjzDo+e8U5EdR8nzKq8Yj1htzLm3b6eQEEam3/rrtg==}
+  /@floating-ui/dom/1.0.8:
+    resolution: {integrity: sha512-uUm1xYQ0xdmFLhtetKgjK9AtF4INwDVwuJZvpK19ENHg1wYn7T0w9phYyCL5+HEpgJtX4ZYG6HJkDbtd2yP6cg==}
     dependencies:
-      '@floating-ui/core': 0.7.3
+      '@floating-ui/core': 1.0.3
     dev: false
 
-  /@humanwhocodes/config-array/0.10.4:
-    resolution: {integrity: sha512-mXAIHxZT3Vcpg83opl1wGlVZ9xydbfZO3r5YfRSH6Gpp2J/PfdBP0wbDa2sO6/qRbcalpoevVyW6A/fI6LfeMw==}
+  /@humanwhocodes/config-array/0.11.7:
+    resolution: {integrity: sha512-kBbPWzN8oVMLb0hOUYXhmxggL/1cJE6ydvjDIGi9EnAGUyA7cLVKQg+d/Dsm+KZwx2czGHrCmMVLiyg8s5JPKw==}
     engines: {node: '>=10.10.0'}
     dependencies:
       '@humanwhocodes/object-schema': 1.2.1
@@ -149,22 +160,23 @@ packages:
       - supports-color
     dev: true
 
-  /@humanwhocodes/gitignore-to-minimatch/1.0.2:
-    resolution: {integrity: sha512-rSqmMJDdLFUsyxR6FMtD00nfQKKLFb1kv+qBbOVKqErvloEIJLo5bDTJTQNTYgeyp78JsA7u/NPi5jT1GR/MuA==}
+  /@humanwhocodes/module-importer/1.0.1:
+    resolution: {integrity: sha512-bxveV4V8v5Yb4ncFTT3rPSgZBOpCkjfK0y4oVVVJwIuDVBRMDXrPyXRL988i5ap9m9bnyEEjWfm5WkBmtffLfA==}
+    engines: {node: '>=12.22'}
     dev: true
 
   /@humanwhocodes/object-schema/1.2.1:
     resolution: {integrity: sha512-ZnQMnLV4e7hDlUvw8H+U8ASL02SS2Gn6+9Ac3wGGLIe7+je2AeAOxPY+izIPJDfFDb7eDjev0Us8MO1iFRN8hA==}
     dev: true
 
-  /@iconify-json/ep/1.1.7:
-    resolution: {integrity: sha512-GhXWVKalXFlrGgfrCXAgqBre5hv3pPAknuxyywmjamcrL5gl5Mq9WOZtuhb4cB6cJ5pMiKOMtegt73FheqWscA==}
+  /@iconify-json/ep/1.1.8:
+    resolution: {integrity: sha512-pHCrsWU1R9/pTDU+Fps4+mjqOQFLtpGdXWegkhQ1P1DlgQAlCPyICtl6E1s8b7VwJMeZXaK84HA02UF6WD0o/Q==}
     dependencies:
-      '@iconify/types': 1.1.0
+      '@iconify/types': 2.0.0
     dev: true
 
-  /@iconify/types/1.1.0:
-    resolution: {integrity: sha512-Jh0llaK2LRXQoYsorIH8maClebsnzTcve+7U3rQUSnC11X4jtPnFuyatqFLvMxZ8MLG8dB4zfHsbPfuvxluONw==}
+  /@iconify/types/2.0.0:
+    resolution: {integrity: sha512-+wluvCrRhXrhyOmRDJ3q8mux9JkKy5SJ/v8ol2tu4FVjyYvtEzkc/3pK15ET6RKg4b4w4BmTk1+gsCUhf21Ykg==}
     dev: true
 
   /@intlify/core-base/9.2.2:
@@ -219,8 +231,8 @@ packages:
     resolution: {integrity: sha512-XPSJHWmi394fuUuzDnGz1wiKqWfo1yXecHQMRf2l6hztTO+nPru658AyDngaBe7isIxEkRsPR3FZh+s7iVa4Uw==}
     dev: true
 
-  /@jridgewell/trace-mapping/0.3.15:
-    resolution: {integrity: sha512-oWZNOULl+UbhsgB51uuZzglikfIKSUBO/M9W2OfEjn7cmqoAiCgmv9lyACTUacZwBz0ITnJ2NqjU8Tx0DHL88g==}
+  /@jridgewell/trace-mapping/0.3.17:
+    resolution: {integrity: sha512-MCNzAp77qzKca9+W/+I0+sEpaUnZoeasnghNeVc41VZCEKaCH73Vq3BZZ/SzWIgrqE4H4ceI+p+b6C0mHf9T4g==}
     dependencies:
       '@jridgewell/resolve-uri': 3.1.0
       '@jridgewell/sourcemap-codec': 1.4.14
@@ -244,7 +256,7 @@ packages:
     engines: {node: '>= 8'}
     dependencies:
       '@nodelib/fs.scandir': 2.1.5
-      fastq: 1.13.0
+      fastq: 1.14.0
     dev: true
 
   /@sxzz/popperjs-es/2.11.7:
@@ -259,11 +271,11 @@ packages:
   /@types/chai-subset/1.3.3:
     resolution: {integrity: sha512-frBecisrNGz+F4T6bcc+NLeolfiojh5FxW2klu669+8BARtyQv2C/GkNW6FUodVe4BroGMP/wER/YDGc7rEllw==}
     dependencies:
-      '@types/chai': 4.3.3
+      '@types/chai': 4.3.4
     dev: true
 
-  /@types/chai/4.3.3:
-    resolution: {integrity: sha512-hC7OMnszpxhZPduX+m+nrx+uFoLkWOMiR4oa/AZF3MuSETYTZmFfJAHqZEM8MVlvfG7BEUcgvtwoCTxBp6hm3g==}
+  /@types/chai/4.3.4:
+    resolution: {integrity: sha512-KnRanxnpfpjUTqTCXslZSEdLfXExwgNxYPdiO2WGUj8+HDjFi8R3k5RVKPeSCzLjCcshCAtVO2QBbVuAV4kTnw==}
     dev: true
 
   /@types/istanbul-lib-coverage/2.0.4:
@@ -277,23 +289,27 @@ packages:
   /@types/lodash-es/4.17.6:
     resolution: {integrity: sha512-R+zTeVUKDdfoRxpAryaQNRKk3105Rrgx2CFRClIgRGaqDTdjsm8h6IYA8ir584W3ePzkZfst5xIgDwYrlh9HLg==}
     dependencies:
-      '@types/lodash': 4.14.183
+      '@types/lodash': 4.14.191
     dev: false
 
-  /@types/lodash/4.14.183:
-    resolution: {integrity: sha512-UXavyuxzXKMqJPEpFPri6Ku5F9af6ZJXUneHhvQJxavrEjuHkFp2YnDWHcxJiG7hk8ZkWqjcyNeW1s/smZv5cw==}
+  /@types/lodash/4.14.191:
+    resolution: {integrity: sha512-BdZ5BCCvho3EIXw6wUCXHe7rS53AIDPLE+JzwgT+OsJk53oBfbSmZZ7CX4VaRoN78N+TJpFi9QPlfIVNmJYWxQ==}
     dev: false
 
-  /@types/node/18.7.6:
-    resolution: {integrity: sha512-EdxgKRXgYsNITy5mjjXjVE/CS8YENSdhiagGrLqjG0pvA2owgJ6i4l7wy/PFZGC0B1/H20lWKN7ONVDNYDZm7A==}
+  /@types/node/18.11.11:
+    resolution: {integrity: sha512-KJ021B1nlQUBLopzZmPBVuGU9un7WJd/W4ya7Ih02B4Uwky5Nja0yGYav2EfYIk0RR2Q9oVhf60S2XR1BCWJ2g==}
+    dev: true
+
+  /@types/semver/7.3.13:
+    resolution: {integrity: sha512-21cFJr9z3g5dW8B0CVI9g2O9beqaThGQ6ZFBqHfwhzLDKUxaqTIy3vnfah/UPkfOiF2pLq+tGz+W8RyCskuslw==}
     dev: true
 
-  /@types/web-bluetooth/0.0.14:
-    resolution: {integrity: sha512-5d2RhCard1nQUC3aHcq/gHzWYO6K0WJmAbjO7mQJgCQKtZpgXxv1rOM6O/dBDhDYYVutk1sciOgNSe+5YyfM8A==}
+  /@types/web-bluetooth/0.0.16:
+    resolution: {integrity: sha512-oh8q2Zc32S6gd/j50GowEjKLoOVOwHP/bWVjKJInBwQqdOYMdPrf1oVlelTlyfFK3CKxL1uahMDAr+vy8T7yMQ==}
     dev: false
 
-  /@typescript-eslint/eslint-plugin/5.33.1_vsoshirnpb7xw6mr7xomgfas2i:
-    resolution: {integrity: sha512-S1iZIxrTvKkU3+m63YUOxYPKaP+yWDQrdhxTglVDVEVBf+aCSw85+BmJnyUaQQsk5TXFG/LpBu9fa+LrAQ91fQ==}
+  /@typescript-eslint/eslint-plugin/5.45.1_tdm6ms4ntwhlpozn7kjqrhum74:
+    resolution: {integrity: sha512-cOizjPlKEh0bXdFrBLTrI/J6B/QMlhwE9auOov53tgB+qMukH6/h8YAK/qw+QJGct/PTbdh2lytGyipxCcEtAw==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     peerDependencies:
       '@typescript-eslint/parser': ^5.0.0
@@ -303,24 +319,24 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@typescript-eslint/parser': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
-      '@typescript-eslint/scope-manager': 5.33.1
-      '@typescript-eslint/type-utils': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
-      '@typescript-eslint/utils': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
+      '@typescript-eslint/parser': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
+      '@typescript-eslint/scope-manager': 5.45.1
+      '@typescript-eslint/type-utils': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
+      '@typescript-eslint/utils': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
       debug: 4.3.4
-      eslint: 8.22.0
-      functional-red-black-tree: 1.0.1
-      ignore: 5.2.0
+      eslint: 8.29.0
+      ignore: 5.2.1
+      natural-compare-lite: 1.4.0
       regexpp: 3.2.0
-      semver: 7.3.7
-      tsutils: 3.21.0_typescript@4.7.4
-      typescript: 4.7.4
+      semver: 7.3.8
+      tsutils: 3.21.0_typescript@4.9.3
+      typescript: 4.9.3
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /@typescript-eslint/parser/5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq:
-    resolution: {integrity: sha512-IgLLtW7FOzoDlmaMoXdxG8HOCByTBXrB1V2ZQYSEV1ggMmJfAkMWTwUjjzagS6OkfpySyhKFkBw7A9jYmcHpZA==}
+  /@typescript-eslint/parser/5.45.1_s5ps7njkmjlaqajutnox5ntcla:
+    resolution: {integrity: sha512-JQ3Ep8bEOXu16q0ztsatp/iQfDCtvap7sp/DKo7DWltUquj5AfCOpX2zSzJ8YkAVnrQNqQ5R62PBz2UtrfmCkA==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: ^6.0.0 || ^7.0.0 || ^8.0.0
@@ -329,26 +345,26 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@typescript-eslint/scope-manager': 5.33.1
-      '@typescript-eslint/types': 5.33.1
-      '@typescript-eslint/typescript-estree': 5.33.1_typescript@4.7.4
+      '@typescript-eslint/scope-manager': 5.45.1
+      '@typescript-eslint/types': 5.45.1
+      '@typescript-eslint/typescript-estree': 5.45.1_typescript@4.9.3
       debug: 4.3.4
-      eslint: 8.22.0
-      typescript: 4.7.4
+      eslint: 8.29.0
+      typescript: 4.9.3
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /@typescript-eslint/scope-manager/5.33.1:
-    resolution: {integrity: sha512-8ibcZSqy4c5m69QpzJn8XQq9NnqAToC8OdH/W6IXPXv83vRyEDPYLdjAlUx8h/rbusq6MkW4YdQzURGOqsn3CA==}
+  /@typescript-eslint/scope-manager/5.45.1:
+    resolution: {integrity: sha512-D6fCileR6Iai7E35Eb4Kp+k0iW7F1wxXYrOhX/3dywsOJpJAQ20Fwgcf+P/TDtvQ7zcsWsrJaglaQWDhOMsspQ==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dependencies:
-      '@typescript-eslint/types': 5.33.1
-      '@typescript-eslint/visitor-keys': 5.33.1
+      '@typescript-eslint/types': 5.45.1
+      '@typescript-eslint/visitor-keys': 5.45.1
     dev: true
 
-  /@typescript-eslint/type-utils/5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq:
-    resolution: {integrity: sha512-X3pGsJsD8OiqhNa5fim41YtlnyiWMF/eKsEZGsHID2HcDqeSC5yr/uLOeph8rNF2/utwuI0IQoAK3fpoxcLl2g==}
+  /@typescript-eslint/type-utils/5.45.1_s5ps7njkmjlaqajutnox5ntcla:
+    resolution: {integrity: sha512-aosxFa+0CoYgYEl3aptLe1svP910DJq68nwEJzyQcrtRhC4BN0tJAvZGAe+D0tzjJmFXe+h4leSsiZhwBa2vrA==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: '*'
@@ -357,22 +373,23 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@typescript-eslint/utils': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
+      '@typescript-eslint/typescript-estree': 5.45.1_typescript@4.9.3
+      '@typescript-eslint/utils': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
       debug: 4.3.4
-      eslint: 8.22.0
-      tsutils: 3.21.0_typescript@4.7.4
-      typescript: 4.7.4
+      eslint: 8.29.0
+      tsutils: 3.21.0_typescript@4.9.3
+      typescript: 4.9.3
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /@typescript-eslint/types/5.33.1:
-    resolution: {integrity: sha512-7K6MoQPQh6WVEkMrMW5QOA5FO+BOwzHSNd0j3+BlBwd6vtzfZceJ8xJ7Um2XDi/O3umS8/qDX6jdy2i7CijkwQ==}
+  /@typescript-eslint/types/5.45.1:
+    resolution: {integrity: sha512-HEW3U0E5dLjUT+nk7b4lLbOherS1U4ap+b9pfu2oGsW3oPu7genRaY9dDv3nMczC1rbnRY2W/D7SN05wYoGImg==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dev: true
 
-  /@typescript-eslint/typescript-estree/5.33.1_typescript@4.7.4:
-    resolution: {integrity: sha512-JOAzJ4pJ+tHzA2pgsWQi4804XisPHOtbvwUyqsuuq8+y5B5GMZs7lI1xDWs6V2d7gE/Ez5bTGojSK12+IIPtXA==}
+  /@typescript-eslint/typescript-estree/5.45.1_typescript@4.9.3:
+    resolution: {integrity: sha512-76NZpmpCzWVrrb0XmYEpbwOz/FENBi+5W7ipVXAsG3OoFrQKJMiaqsBMbvGRyLtPotGqUfcY7Ur8j0dksDJDng==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     peerDependencies:
       typescript: '*'
@@ -380,60 +397,62 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@typescript-eslint/types': 5.33.1
-      '@typescript-eslint/visitor-keys': 5.33.1
+      '@typescript-eslint/types': 5.45.1
+      '@typescript-eslint/visitor-keys': 5.45.1
       debug: 4.3.4
       globby: 11.1.0
       is-glob: 4.0.3
-      semver: 7.3.7
-      tsutils: 3.21.0_typescript@4.7.4
-      typescript: 4.7.4
+      semver: 7.3.8
+      tsutils: 3.21.0_typescript@4.9.3
+      typescript: 4.9.3
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /@typescript-eslint/utils/5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq:
-    resolution: {integrity: sha512-uphZjkMaZ4fE8CR4dU7BquOV6u0doeQAr8n6cQenl/poMaIyJtBu8eys5uk6u5HiDH01Mj5lzbJ5SfeDz7oqMQ==}
+  /@typescript-eslint/utils/5.45.1_s5ps7njkmjlaqajutnox5ntcla:
+    resolution: {integrity: sha512-rlbC5VZz68+yjAzQBc4I7KDYVzWG2X/OrqoZrMahYq3u8FFtmQYc+9rovo/7wlJH5kugJ+jQXV5pJMnofGmPRw==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: ^6.0.0 || ^7.0.0 || ^8.0.0
     dependencies:
       '@types/json-schema': 7.0.11
-      '@typescript-eslint/scope-manager': 5.33.1
-      '@typescript-eslint/types': 5.33.1
-      '@typescript-eslint/typescript-estree': 5.33.1_typescript@4.7.4
-      eslint: 8.22.0
+      '@types/semver': 7.3.13
+      '@typescript-eslint/scope-manager': 5.45.1
+      '@typescript-eslint/types': 5.45.1
+      '@typescript-eslint/typescript-estree': 5.45.1_typescript@4.9.3
+      eslint: 8.29.0
       eslint-scope: 5.1.1
-      eslint-utils: 3.0.0_eslint@8.22.0
+      eslint-utils: 3.0.0_eslint@8.29.0
+      semver: 7.3.8
     transitivePeerDependencies:
       - supports-color
       - typescript
     dev: true
 
-  /@typescript-eslint/visitor-keys/5.33.1:
-    resolution: {integrity: sha512-nwIxOK8Z2MPWltLKMLOEZwmfBZReqUdbEoHQXeCpa+sRVARe5twpJGHCB4dk9903Yaf0nMAlGbQfaAH92F60eg==}
+  /@typescript-eslint/visitor-keys/5.45.1:
+    resolution: {integrity: sha512-cy9ln+6rmthYWjH9fmx+5FU/JDpjQb586++x2FZlveq7GdGuLLW9a2Jcst2TGekH82bXpfmRNSwP9tyEs6RjvQ==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dependencies:
-      '@typescript-eslint/types': 5.33.1
+      '@typescript-eslint/types': 5.45.1
       eslint-visitor-keys: 3.3.0
     dev: true
 
-  /@vitejs/plugin-vue/3.0.3_vite@3.0.8+vue@3.2.37:
-    resolution: {integrity: sha512-U4zNBlz9mg+TA+i+5QPc3N5lQvdUXENZLO2h0Wdzp56gI1MWhqJOv+6R+d4kOzoaSSq6TnGPBdZAXKOe4lXy6g==}
+  /@vitejs/plugin-vue/3.2.0_vite@3.2.5+vue@3.2.45:
+    resolution: {integrity: sha512-E0tnaL4fr+qkdCNxJ+Xd0yM31UwMkQje76fsDVBBUCoGOUPexu2VDUYHL8P4CwV+zMvWw6nlRw19OnRKmYAJpw==}
     engines: {node: ^14.18.0 || >=16.0.0}
     peerDependencies:
       vite: ^3.0.0
       vue: ^3.2.25
     dependencies:
-      vite: 3.0.8_sass@1.54.4
-      vue: 3.2.37
+      vite: 3.2.5_tr7neyh64qpc4cjqb6wcsya4ri
+      vue: 3.2.45
     dev: true
 
-  /@vitest/coverage-c8/0.22.0_jsdom@20.0.0+sass@1.54.4:
-    resolution: {integrity: sha512-jwW6b8U+h9nbzQfKoRmpf2xjDg+mcAjLIdVUrZGhjTnIdekGfvoqFoeiXzsLv2HwYBeFi4943lYUftuj8qD1FQ==}
+  /@vitest/coverage-c8/0.22.1_jsdom@20.0.3+sass@1.56.1:
+    resolution: {integrity: sha512-KOOYpO7EGpaF+nD8GD+Y05D0JtZp12NUu6DdLXvBPqSOPo2HkZ7KNBtfR0rb6gOy3NLtGiWTYTzCwhajgb2HlA==}
     dependencies:
       c8: 7.12.0
-      vitest: 0.22.0_jsdom@20.0.0+sass@1.54.4
+      vitest: 0.22.1_jsdom@20.0.3+sass@1.56.1
     transitivePeerDependencies:
       - '@edge-runtime/vm'
       - '@vitest/browser'
@@ -443,6 +462,7 @@ packages:
       - less
       - sass
       - stylus
+      - sugarss
       - supports-color
       - terser
     dev: true
@@ -462,9 +482,9 @@ packages:
     dependencies:
       '@volar/code-gen': 0.38.9
       '@volar/source-map': 0.38.9
-      '@vue/compiler-core': 3.2.37
-      '@vue/compiler-dom': 3.2.37
-      '@vue/shared': 3.2.37
+      '@vue/compiler-core': 3.2.45
+      '@vue/compiler-dom': 3.2.45
+      '@vue/shared': 3.2.45
     dev: true
 
   /@volar/vue-typescript/0.38.9:
@@ -473,62 +493,62 @@ packages:
       '@volar/code-gen': 0.38.9
       '@volar/source-map': 0.38.9
       '@volar/vue-code-gen': 0.38.9
-      '@vue/compiler-sfc': 3.2.37
-      '@vue/reactivity': 3.2.37
+      '@vue/compiler-sfc': 3.2.45
+      '@vue/reactivity': 3.2.45
     dev: true
 
-  /@vue/compiler-core/3.2.37:
-    resolution: {integrity: sha512-81KhEjo7YAOh0vQJoSmAD68wLfYqJvoiD4ulyedzF+OEk/bk6/hx3fTNVfuzugIIaTrOx4PGx6pAiBRe5e9Zmg==}
+  /@vue/compiler-core/3.2.45:
+    resolution: {integrity: sha512-rcMj7H+PYe5wBV3iYeUgbCglC+pbpN8hBLTJvRiK2eKQiWqu+fG9F+8sW99JdL4LQi7Re178UOxn09puSXvn4A==}
     dependencies:
-      '@babel/parser': 7.18.11
-      '@vue/shared': 3.2.37
+      '@babel/parser': 7.20.5
+      '@vue/shared': 3.2.45
       estree-walker: 2.0.2
       source-map: 0.6.1
 
-  /@vue/compiler-dom/3.2.37:
-    resolution: {integrity: sha512-yxJLH167fucHKxaqXpYk7x8z7mMEnXOw3G2q62FTkmsvNxu4FQSu5+3UMb+L7fjKa26DEzhrmCxAgFLLIzVfqQ==}
+  /@vue/compiler-dom/3.2.45:
+    resolution: {integrity: sha512-tyYeUEuKqqZO137WrZkpwfPCdiiIeXYCcJ8L4gWz9vqaxzIQRccTSwSWZ/Axx5YR2z+LvpUbmPNXxuBU45lyRw==}
     dependencies:
-      '@vue/compiler-core': 3.2.37
-      '@vue/shared': 3.2.37
+      '@vue/compiler-core': 3.2.45
+      '@vue/shared': 3.2.45
 
-  /@vue/compiler-sfc/3.2.37:
-    resolution: {integrity: sha512-+7i/2+9LYlpqDv+KTtWhOZH+pa8/HnX/905MdVmAcI/mPQOBwkHHIzrsEsucyOIZQYMkXUiTkmZq5am/NyXKkg==}
+  /@vue/compiler-sfc/3.2.45:
+    resolution: {integrity: sha512-1jXDuWah1ggsnSAOGsec8cFjT/K6TMZ0sPL3o3d84Ft2AYZi2jWJgRMjw4iaK0rBfA89L5gw427H4n1RZQBu6Q==}
     dependencies:
-      '@babel/parser': 7.18.11
-      '@vue/compiler-core': 3.2.37
-      '@vue/compiler-dom': 3.2.37
-      '@vue/compiler-ssr': 3.2.37
-      '@vue/reactivity-transform': 3.2.37
-      '@vue/shared': 3.2.37
+      '@babel/parser': 7.20.5
+      '@vue/compiler-core': 3.2.45
+      '@vue/compiler-dom': 3.2.45
+      '@vue/compiler-ssr': 3.2.45
+      '@vue/reactivity-transform': 3.2.45
+      '@vue/shared': 3.2.45
       estree-walker: 2.0.2
       magic-string: 0.25.9
-      postcss: 8.4.16
+      postcss: 8.4.19
       source-map: 0.6.1
 
-  /@vue/compiler-ssr/3.2.37:
-    resolution: {integrity: sha512-7mQJD7HdXxQjktmsWp/J67lThEIcxLemz1Vb5I6rYJHR5vI+lON3nPGOH3ubmbvYGt8xEUaAr1j7/tIFWiEOqw==}
+  /@vue/compiler-ssr/3.2.45:
+    resolution: {integrity: sha512-6BRaggEGqhWht3lt24CrIbQSRD5O07MTmd+LjAn5fJj568+R9eUD2F7wMQJjX859seSlrYog7sUtrZSd7feqrQ==}
     dependencies:
-      '@vue/compiler-dom': 3.2.37
-      '@vue/shared': 3.2.37
+      '@vue/compiler-dom': 3.2.45
+      '@vue/shared': 3.2.45
 
-  /@vue/devtools-api/6.2.1:
-    resolution: {integrity: sha512-OEgAMeQXvCoJ+1x8WyQuVZzFo0wcyCmUR3baRVLmKBo1LmYZWMlRiXlux5jd0fqVJu6PfDbOrZItVqUEzLobeQ==}
+  /@vue/devtools-api/6.4.5:
+    resolution: {integrity: sha512-JD5fcdIuFxU4fQyXUu3w2KpAJHzTVdN+p4iOX2lMWSHMOoQdMAcpFLZzm9Z/2nmsoZ1a96QEhZ26e50xLBsgOQ==}
     dev: false
 
-  /@vue/eslint-config-prettier/7.0.0_2xd4q2tc5cqa5as7uugqhp6oue:
+  /@vue/eslint-config-prettier/7.0.0_yz7jvh5l5mgl5xhpwlgtkgcccy:
     resolution: {integrity: sha512-/CTc6ML3Wta1tCe1gUeO0EYnVXfo3nJXsIhZ8WJr3sov+cGASr6yuiibJTL6lmIBm7GobopToOuB3B6AWyV0Iw==}
     peerDependencies:
       eslint: '>= 7.28.0'
       prettier: '>= 2.0.0'
     dependencies:
-      eslint: 8.22.0
-      eslint-config-prettier: 8.5.0_eslint@8.22.0
-      eslint-plugin-prettier: 4.2.1_i2cojdczqdiurzgttlwdgf764e
-      prettier: 2.7.1
+      eslint: 8.29.0
+      eslint-config-prettier: 8.5.0_eslint@8.29.0
+      eslint-plugin-prettier: 4.2.1_nrhoyyjffvfyk4vtlt5destxgm
+      prettier: 2.8.0
     dev: true
 
-  /@vue/eslint-config-typescript/11.0.0_4py5zxx5ck6utobkmozwvrmyiy:
-    resolution: {integrity: sha512-txuRzxnQVmtUvvy9UyWUy9sHWXNeRPGmSPqP53hRtaiUeCTAondI9Ho9GQYI/8/eWljYOST7iA4Aa8sANBkWaA==}
+  /@vue/eslint-config-typescript/11.0.2_sb3redb42nwkq4q6vnqovjecl4:
+    resolution: {integrity: sha512-EiKud1NqlWmSapBFkeSrE994qpKx7/27uCGnhdqzllYDpQZroyX/O6bwjEpeuyKamvLbsGdO6PMR2faIf+zFnw==}
     engines: {node: ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: ^6.2.0 || ^7.0.0 || ^8.0.0
@@ -538,132 +558,114 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@typescript-eslint/eslint-plugin': 5.33.1_vsoshirnpb7xw6mr7xomgfas2i
-      '@typescript-eslint/parser': 5.33.1_4rv7y5c6xz3vfxwhbrcxxi73bq
-      eslint: 8.22.0
-      eslint-plugin-vue: 9.3.0_eslint@8.22.0
-      typescript: 4.7.4
-      vue-eslint-parser: 9.0.3_eslint@8.22.0
+      '@typescript-eslint/eslint-plugin': 5.45.1_tdm6ms4ntwhlpozn7kjqrhum74
+      '@typescript-eslint/parser': 5.45.1_s5ps7njkmjlaqajutnox5ntcla
+      eslint: 8.29.0
+      eslint-plugin-vue: 9.8.0_eslint@8.29.0
+      typescript: 4.9.3
+      vue-eslint-parser: 9.1.0_eslint@8.29.0
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /@vue/reactivity-transform/3.2.37:
-    resolution: {integrity: sha512-IWopkKEb+8qpu/1eMKVeXrK0NLw9HicGviJzhJDEyfxTR9e1WtpnnbYkJWurX6WwoFP0sz10xQg8yL8lgskAZg==}
+  /@vue/reactivity-transform/3.2.45:
+    resolution: {integrity: sha512-BHVmzYAvM7vcU5WmuYqXpwaBHjsS8T63jlKGWVtHxAHIoMIlmaMyurUSEs1Zcg46M4AYT5MtB1U274/2aNzjJQ==}
     dependencies:
-      '@babel/parser': 7.18.11
-      '@vue/compiler-core': 3.2.37
-      '@vue/shared': 3.2.37
+      '@babel/parser': 7.20.5
+      '@vue/compiler-core': 3.2.45
+      '@vue/shared': 3.2.45
       estree-walker: 2.0.2
       magic-string: 0.25.9
 
-  /@vue/reactivity/3.2.37:
-    resolution: {integrity: sha512-/7WRafBOshOc6m3F7plwzPeCu/RCVv9uMpOwa/5PiY1Zz+WLVRWiy0MYKwmg19KBdGtFWsmZ4cD+LOdVPcs52A==}
+  /@vue/reactivity/3.2.45:
+    resolution: {integrity: sha512-PRvhCcQcyEVohW0P8iQ7HDcIOXRjZfAsOds3N99X/Dzewy8TVhTCT4uXpAHfoKjVTJRA0O0K+6QNkDIZAxNi3A==}
     dependencies:
-      '@vue/shared': 3.2.37
+      '@vue/shared': 3.2.45
 
-  /@vue/runtime-core/3.2.37:
-    resolution: {integrity: sha512-JPcd9kFyEdXLl/i0ClS7lwgcs0QpUAWj+SKX2ZC3ANKi1U4DOtiEr6cRqFXsPwY5u1L9fAjkinIdB8Rz3FoYNQ==}
+  /@vue/runtime-core/3.2.45:
+    resolution: {integrity: sha512-gzJiTA3f74cgARptqzYswmoQx0fIA+gGYBfokYVhF8YSXjWTUA2SngRzZRku2HbGbjzB6LBYSbKGIaK8IW+s0A==}
     dependencies:
-      '@vue/reactivity': 3.2.37
-      '@vue/shared': 3.2.37
+      '@vue/reactivity': 3.2.45
+      '@vue/shared': 3.2.45
 
-  /@vue/runtime-dom/3.2.37:
-    resolution: {integrity: sha512-HimKdh9BepShW6YozwRKAYjYQWg9mQn63RGEiSswMbW+ssIht1MILYlVGkAGGQbkhSh31PCdoUcfiu4apXJoPw==}
+  /@vue/runtime-dom/3.2.45:
+    resolution: {integrity: sha512-cy88YpfP5Ue2bDBbj75Cb4bIEZUMM/mAkDMfqDTpUYVgTf/kuQ2VQ8LebuZ8k6EudgH8pYhsGWHlY0lcxlvTwA==}
     dependencies:
-      '@vue/runtime-core': 3.2.37
-      '@vue/shared': 3.2.37
-      csstype: 2.6.20
+      '@vue/runtime-core': 3.2.45
+      '@vue/shared': 3.2.45
+      csstype: 2.6.21
 
-  /@vue/server-renderer/3.2.37_vue@3.2.37:
-    resolution: {integrity: sha512-kLITEJvaYgZQ2h47hIzPh2K3jG8c1zCVbp/o/bzQOyvzaKiCquKS7AaioPI28GNxIsE/zSx+EwWYsNxDCX95MA==}
+  /@vue/server-renderer/3.2.45_vue@3.2.45:
+    resolution: {integrity: sha512-ebiMq7q24WBU1D6uhPK//2OTR1iRIyxjF5iVq/1a5I1SDMDyDu4Ts6fJaMnjrvD3MqnaiFkKQj+LKAgz5WIK3g==}
     peerDependencies:
-      vue: 3.2.37
+      vue: 3.2.45
     dependencies:
-      '@vue/compiler-ssr': 3.2.37
-      '@vue/shared': 3.2.37
-      vue: 3.2.37
+      '@vue/compiler-ssr': 3.2.45
+      '@vue/shared': 3.2.45
+      vue: 3.2.45
 
-  /@vue/shared/3.2.37:
-    resolution: {integrity: sha512-4rSJemR2NQIo9Klm1vabqWjD8rs/ZaJSzMxkMNeJS6lHiUjjUeYFbooN19NgFjztubEKh3WlZUeOLVdbbUWHsw==}
+  /@vue/shared/3.2.45:
+    resolution: {integrity: sha512-Ewzq5Yhimg7pSztDV+RH1UDKBzmtqieXQlpTVm2AwraoRL/Rks96mvd8Vgi7Lj+h+TH8dv7mXD3FRZR3TUvbSg==}
 
-  /@vue/test-utils/2.0.2_vue@3.2.37:
-    resolution: {integrity: sha512-E2P4oXSaWDqTZNbmKZFVLrNN/siVN78YkEqs7pHryWerrlZR9bBFLWdJwRoguX45Ru6HxIflzKl4vQvwRMwm5g==}
+  /@vue/test-utils/2.2.6_vue@3.2.45:
+    resolution: {integrity: sha512-64zHtJZdG7V/U2L0j/z3Pt5bSygccI3xs+Kl7LB73AZK4MQ8WONJhqDQPK8leUFFA9CmmoJygeky7zcl2hX10A==}
     peerDependencies:
       vue: ^3.0.1
     dependencies:
-      vue: 3.2.37
+      vue: 3.2.45
     dev: true
 
-  /@vueuse/core/8.9.4_vue@3.2.37:
-    resolution: {integrity: sha512-B/Mdj9TK1peFyWaPof+Zf/mP9XuGAngaJZBwPaXBvU3aCTZlx3ltlrFFFyMV4iGBwsjSCeUCgZrtkEj9dS2Y3Q==}
-    peerDependencies:
-      '@vue/composition-api': ^1.1.0
-      vue: ^2.6.0 || ^3.2.0
-    peerDependenciesMeta:
-      '@vue/composition-api':
-        optional: true
-      vue:
-        optional: true
+  /@vueuse/core/9.6.0_vue@3.2.45:
+    resolution: {integrity: sha512-qGUcjKQXHgN+jqXEgpeZGoxdCbIDCdVPz3QiF1uyecVGbMuM63o96I1GjYx5zskKgRI0FKSNsVWM7rwrRMTf6A==}
     dependencies:
-      '@types/web-bluetooth': 0.0.14
-      '@vueuse/metadata': 8.9.4
-      '@vueuse/shared': 8.9.4_vue@3.2.37
-      vue: 3.2.37
-      vue-demi: 0.13.8_vue@3.2.37
+      '@types/web-bluetooth': 0.0.16
+      '@vueuse/metadata': 9.6.0
+      '@vueuse/shared': 9.6.0_vue@3.2.45
+      vue-demi: 0.13.11_vue@3.2.45
+    transitivePeerDependencies:
+      - '@vue/composition-api'
+      - vue
     dev: false
 
-  /@vueuse/metadata/8.9.4:
-    resolution: {integrity: sha512-IwSfzH80bnJMzqhaapqJl9JRIiyQU0zsRGEgnxN6jhq7992cPUJIRfV+JHRIZXjYqbwt07E1gTEp0R0zPJ1aqw==}
+  /@vueuse/metadata/9.6.0:
+    resolution: {integrity: sha512-sIC8R+kWkIdpi5X2z2Gk8TRYzmczDwHRhEFfCu2P+XW2JdPoXrziqsGpDDsN7ykBx4ilwieS7JUIweVGhvZ93w==}
     dev: false
 
-  /@vueuse/shared/8.9.4_vue@3.2.37:
-    resolution: {integrity: sha512-wt+T30c4K6dGRMVqPddexEVLa28YwxW5OFIPmzUHICjphfAuBFTTdDoyqREZNDOFJZ44ARH1WWQNCUK8koJ+Ag==}
-    peerDependencies:
-      '@vue/composition-api': ^1.1.0
-      vue: ^2.6.0 || ^3.2.0
-    peerDependenciesMeta:
-      '@vue/composition-api':
-        optional: true
-      vue:
-        optional: true
+  /@vueuse/shared/9.6.0_vue@3.2.45:
+    resolution: {integrity: sha512-/eDchxYYhkHnFyrb00t90UfjCx94kRHxc7J1GtBCqCG4HyPMX+krV9XJgVtWIsAMaxKVU4fC8NSUviG1JkwhUQ==}
     dependencies:
-      vue: 3.2.37
-      vue-demi: 0.13.8_vue@3.2.37
+      vue-demi: 0.13.11_vue@3.2.45
+    transitivePeerDependencies:
+      - '@vue/composition-api'
+      - vue
     dev: false
 
   /abab/2.0.6:
     resolution: {integrity: sha512-j2afSsaIENvHZN2B8GOpF566vZ5WVk5opAiMTvWgaQT8DkbOqsTfvNAvHoRGU2zzP8cPoqys+xHTRDWW8L+/BA==}
     dev: true
 
-  /acorn-globals/6.0.0:
-    resolution: {integrity: sha512-ZQl7LOWaF5ePqqcX4hLuv/bLXYQNfNWw2c0/yX/TsPRKamzHcTGQnlCjHT3TsmkOUVEPS3crCxiPfdzE/Trlhg==}
+  /acorn-globals/7.0.1:
+    resolution: {integrity: sha512-umOSDSDrfHbTNPuNpC2NSnnA3LUrqpevPb4T9jRx4MagXNS0rs+gwiTcAvqCRmsD6utzsrzNt+ebm00SNWiC3Q==}
     dependencies:
-      acorn: 7.4.1
-      acorn-walk: 7.2.0
+      acorn: 8.8.1
+      acorn-walk: 8.2.0
     dev: true
 
-  /acorn-jsx/5.3.2_acorn@8.8.0:
+  /acorn-jsx/5.3.2_acorn@8.8.1:
     resolution: {integrity: sha512-rq9s+JNhf0IChjtDXxllJ7g41oZk5SlXtp0LHwyA5cejwn7vKmKp4pPri6YEePv2PU65sAsegbXtIinmDFDXgQ==}
     peerDependencies:
       acorn: ^6.0.0 || ^7.0.0 || ^8.0.0
     dependencies:
-      acorn: 8.8.0
-    dev: true
-
-  /acorn-walk/7.2.0:
-    resolution: {integrity: sha512-OPdCF6GsMIP+Az+aWfAAOEt2/+iVDKE7oy6lJ098aoe59oAmK76qV6Gw60SbZ8jHuG2wH058GF4pLFbYamYrVA==}
-    engines: {node: '>=0.4.0'}
+      acorn: 8.8.1
     dev: true
 
-  /acorn/7.4.1:
-    resolution: {integrity: sha512-nQyp0o1/mNdbTO1PO6kHkwSrmgZ0MT/jCCpNiwbUjGoRN4dlBhqJtoQuCnEOKzgTVwg0ZWiCoQy6SxMebQVh8A==}
+  /acorn-walk/8.2.0:
+    resolution: {integrity: sha512-k+iyHEuPgSw6SbuDpGQM+06HQUa04DZ3o+F6CSzXMvvI5KMvnaEqXe+YVe555R9nn6GPt404fos4wcgpw12SDA==}
     engines: {node: '>=0.4.0'}
-    hasBin: true
     dev: true
 
-  /acorn/8.8.0:
-    resolution: {integrity: sha512-QOxyigPVrpZ2GXT+PFyZTl6TtOFc5egxHIP9IlQ+RbupQuX4RkT/Bee4/kQuC02Xkzg84JcT7oLYtDIQxp+v7w==}
+  /acorn/8.8.1:
+    resolution: {integrity: sha512-7zFpHzhnqYKrkYdUjF1HI1bzd0VygEGX8lFk4k5zVMqHEoES+P+7TKI+EvLO9WVMJ8eekdO0aDEK044xTXwPPA==}
     engines: {node: '>=0.4.0'}
     hasBin: true
     dev: true
@@ -698,8 +700,8 @@ packages:
       color-convert: 2.0.1
     dev: true
 
-  /anymatch/3.1.2:
-    resolution: {integrity: sha512-P43ePfOAIupkguHUycrc4qJ9kz8ZiuOUijaETwX7THt0Y/GNK7v0aa8rY816xWjZ7rJdA5XdMcpVFTKMq+RvWg==}
+  /anymatch/3.1.3:
+    resolution: {integrity: sha512-KMReFUr0B4t+D+OBkjR3KYqvocp2XaSzO55UcB6mgQMd3KbcE+mWTyvVV7D/zsdEbNnV6acZUutkiHQXvTr1Rw==}
     engines: {node: '>= 8'}
     dependencies:
       normalize-path: 3.0.0
@@ -729,7 +731,7 @@ packages:
   /axios/0.27.2:
     resolution: {integrity: sha512-t+yRIyySRTp/wua5xEr+z1q60QmLq8ABsS5O9Me1AsE5dfKqgnCFzwiCZZ/cGNd1lq4/7akDWMxdhVlucjmnOQ==}
     dependencies:
-      follow-redirects: 1.15.1
+      follow-redirects: 1.15.2
       form-data: 4.0.0
     transitivePeerDependencies:
       - debug
@@ -762,10 +764,6 @@ packages:
       fill-range: 7.0.1
     dev: true
 
-  /browser-process-hrtime/1.0.0:
-    resolution: {integrity: sha512-9o5UecI3GhkpM6DrXr69PblIuWxPKk9Y0jHBRhdocZ2y7YECBFCsHm79Pr3OyR2AvjhDkabFJaDJMYRazHgsow==}
-    dev: true
-
   /c8/7.12.0:
     resolution: {integrity: sha512-CtgQrHOkyxr5koX1wEUmN/5cfDa2ckbHRA4Gy5LAL0zaCFtVWJS5++n+w4/sr2GWGerBxgTjpKeDclk/Qk6W/A==}
     engines: {node: '>=10.12.0'}
@@ -790,15 +788,15 @@ packages:
     engines: {node: '>=6'}
     dev: true
 
-  /chai/4.3.6:
-    resolution: {integrity: sha512-bbcp3YfHCUzMOvKqsztczerVgBKSsEijCySNlHHbX3VG1nskvqjz5Rfso1gGwD6w6oOV3eI60pKuMOV5MV7p3Q==}
+  /chai/4.3.7:
+    resolution: {integrity: sha512-HLnAzZ2iupm25PlN0xFreAlBA5zaBSv3og0DdeGA4Ar6h6rJ3A0rolRUKJhSF2V10GZKDgWF/VmAEsNWjCRB+A==}
     engines: {node: '>=4'}
     dependencies:
       assertion-error: 1.1.0
       check-error: 1.0.2
-      deep-eql: 3.0.1
+      deep-eql: 4.1.3
       get-func-name: 2.0.0
-      loupe: 2.3.4
+      loupe: 2.3.6
       pathval: 1.1.1
       type-detect: 4.0.8
     dev: true
@@ -819,7 +817,7 @@ packages:
     resolution: {integrity: sha512-Dr3sfKRP6oTcjf2JmUmFJfeVMvXBdegxB0iVQ5eb2V10uFJUCAS8OByZdVAyVb8xXNz3GjjTgj9kLWsZTqE6kw==}
     engines: {node: '>= 8.10.0'}
     dependencies:
-      anymatch: 3.1.2
+      anymatch: 3.1.3
       braces: 3.0.2
       glob-parent: 5.1.2
       is-binary-path: 2.1.0
@@ -859,10 +857,8 @@ packages:
     resolution: {integrity: sha512-/Srv4dswyQNBfohGpz9o6Yb3Gz3SrUDqBH5rTuhGR7ahtlbYKnVxw2bCFMRljaA7EXHaXZ8wsHdodFvbkhKmqg==}
     dev: true
 
-  /convert-source-map/1.8.0:
-    resolution: {integrity: sha512-+OQdjP49zViI/6i7nIJpA8rAl4sV/JdPfU9nZs3VqOwGIgizICvuN2ru6fMd+4llL0tar18UYJXfZ/TWtmhUjA==}
-    dependencies:
-      safe-buffer: 5.1.2
+  /convert-source-map/1.9.0:
+    resolution: {integrity: sha512-ASFBup0Mz1uyiIjANan1jzLQami9z1PoYSZCiiYW2FczPbenXc45FZdBZLzOT+r6+iciuEModtmCti+hjaAk0A==}
     dev: true
 
   /cross-spawn/7.0.3:
@@ -895,8 +891,8 @@ packages:
       cssom: 0.3.8
     dev: true
 
-  /csstype/2.6.20:
-    resolution: {integrity: sha512-/WwNkdXfckNgw6S5R125rrW8ez139lBHWouiBvX8dfMFtcn6V81REDqnH7+CRpRipfYlyU1CmOnOxrmGcFOjeA==}
+  /csstype/2.6.21:
+    resolution: {integrity: sha512-Z1PhmomIfypOpoMjRQB70jfvy/wxT50qW08YXO5lMIJkrdq4yOTR+AW7FqutScmB9NkLwxo+jU+kZLbofZZq/w==}
 
   /data-urls/3.0.2:
     resolution: {integrity: sha512-Jy/tj3ldjZJo63sVAvg6LHt2mHvl4V6AgRAmNDtLdm7faqtsx+aJG42rsyCo9JCoRVKwPFzKlIPx3DIibwSIaQ==}
@@ -907,8 +903,13 @@ packages:
       whatwg-url: 11.0.0
     dev: true
 
-  /dayjs/1.11.5:
-    resolution: {integrity: sha512-CAdX5Q3YW3Gclyo5Vpqkgpj8fSdLQcRuzfX6mC6Phy0nfJ0eGYOeS7m4mt2plDWLAtA4TqTakvbboHvUxfe4iA==}
+  /date-fns/2.29.3:
+    resolution: {integrity: sha512-dDCnyH2WnnKusqvZZ6+jA1O51Ibt8ZMRNkDZdyAyK4YfbDwa/cEmuztzG5pk6hqlp9aSBPYcjOlktquahGwGeA==}
+    engines: {node: '>=0.11'}
+    dev: false
+
+  /dayjs/1.11.7:
+    resolution: {integrity: sha512-+Yw9U6YO5TQohxLcIkrXBeY73WP3ejHWVvx8XCk3gxvQDCTEmS48ZrSZCKciI7Bhl/uCMyxYtE9UqRILmFphkQ==}
     dev: false
 
   /debug/4.3.4:
@@ -923,13 +924,13 @@ packages:
       ms: 2.1.2
     dev: true
 
-  /decimal.js/10.4.0:
-    resolution: {integrity: sha512-Nv6ENEzyPQ6AItkGwLE2PGKinZZ9g59vSh2BeH6NqPu0OTKZ5ruJsVqh/orbAnqXc9pBbgXAIrc2EyaCj8NpGg==}
+  /decimal.js/10.4.3:
+    resolution: {integrity: sha512-VBBaLc1MgL5XpzgIP7ny5Z6Nx3UrRkIViUkPUdtl9aya5amy3De1gsUUSB1g3+3sExYNjCAsAznmukyxCb1GRA==}
     dev: true
 
-  /deep-eql/3.0.1:
-    resolution: {integrity: sha512-+QeIQyN5ZuO+3Uk5DYh6/1eKO0m0YmJFGNmFHGACpf1ClL1nmlV/p4gNgbl2pJGxgXb4faqo6UE+M5ACEMyVcw==}
-    engines: {node: '>=0.12'}
+  /deep-eql/4.1.3:
+    resolution: {integrity: sha512-WaEtAOpRA1MQ0eohqZjpGD8zdI0Ovsm8mmFhaDN8dvDZzyoUMcYDnf5Y6iu7HTXxf8JDS23qWa4a+hKCDyOPzw==}
+    engines: {node: '>=6'}
     dependencies:
       type-detect: 4.0.8
     dev: true
@@ -963,27 +964,27 @@ packages:
       webidl-conversions: 7.0.0
     dev: true
 
-  /element-plus/2.2.13_vue@3.2.37:
-    resolution: {integrity: sha512-dKQ7BPZC8deUPhv+6s4GgOL0GyGj3KpUarywxm6s1nWnHjH6FqeZlUcxPqBvJd7W/d81POayx3B13GP+rfkG9g==}
+  /element-plus/2.2.26_vue@3.2.45:
+    resolution: {integrity: sha512-O/rdY5m9DkclpVg8r3GynyqCunm7MxSR142xSsjrZA77bi7bcwA3SIy6SPEDqHi5R4KqgkGYgKSp4Q4e3irbYg==}
     peerDependencies:
       vue: ^3.2.0
     dependencies:
-      '@ctrl/tinycolor': 3.4.1
-      '@element-plus/icons-vue': 2.0.9_vue@3.2.37
-      '@floating-ui/dom': 0.5.4
+      '@ctrl/tinycolor': 3.5.0
+      '@element-plus/icons-vue': 2.0.10_vue@3.2.45
+      '@floating-ui/dom': 1.0.8
       '@popperjs/core': /@sxzz/popperjs-es/2.11.7
-      '@types/lodash': 4.14.183
+      '@types/lodash': 4.14.191
       '@types/lodash-es': 4.17.6
-      '@vueuse/core': 8.9.4_vue@3.2.37
+      '@vueuse/core': 9.6.0_vue@3.2.45
       async-validator: 4.2.5
-      dayjs: 1.11.5
+      dayjs: 1.11.7
       escape-html: 1.0.3
       lodash: 4.17.21
       lodash-es: 4.17.21
-      lodash-unified: 1.0.2_3ib2ivapxullxkx3xftsimdk7u
+      lodash-unified: 1.0.3_3ib2ivapxullxkx3xftsimdk7u
       memoize-one: 6.0.0
       normalize-wheel-es: 1.2.0
-      vue: 3.2.37
+      vue: 3.2.45
     transitivePeerDependencies:
       - '@vue/composition-api'
     dev: false
@@ -992,13 +993,13 @@ packages:
     resolution: {integrity: sha512-MSjYzcWNOA0ewAHpz0MxpYFvwg6yjy1NG3xteoqz644VCo/RPgnr1/GGt+ic3iJTzQ8Eu3TdM14SawnVUmGE6A==}
     dev: true
 
-  /entities/4.3.1:
-    resolution: {integrity: sha512-o4q/dYJlmyjP2zfnaWDUC6A3BQFmVTX+tZPezK7k0GLSU9QYCauscf5Y+qcEPzKL+EixVouYDgLQK5H9GrLpkg==}
+  /entities/4.4.0:
+    resolution: {integrity: sha512-oYp7156SP8LkeGD0GF85ad1X9Ai79WtRsZ2gxJqtBuzH+98YUV6jkHEKlZkMbcrjJjIVJNIDP/3WL9wQkoPbWA==}
     engines: {node: '>=0.12'}
     dev: true
 
-  /esbuild-android-64/0.14.54:
-    resolution: {integrity: sha512-Tz2++Aqqz0rJ7kYBfz+iqyE3QMycD4vk7LBRyWaAVFgFtQ/O8EJOnVmTOiDWYZ/uYzB4kvP+bqejYdVKzE5lAQ==}
+  /esbuild-android-64/0.15.18:
+    resolution: {integrity: sha512-wnpt3OXRhcjfIDSZu9bnzT4/TNTDsOUvip0foZOUBG7QbSt//w3QV4FInVJxNhKc/ErhUxc5z4QjHtMi7/TbgA==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [android]
@@ -1006,8 +1007,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-android-arm64/0.14.54:
-    resolution: {integrity: sha512-F9E+/QDi9sSkLaClO8SOV6etqPd+5DgJje1F9lOWoNncDdOBL2YF59IhsWATSt0TLZbYCf3pNlTHvVV5VfHdvg==}
+  /esbuild-android-arm64/0.15.18:
+    resolution: {integrity: sha512-G4xu89B8FCzav9XU8EjsXacCKSG2FT7wW9J6hOc18soEHJdtWu03L3TQDGf0geNxfLTtxENKBzMSq9LlbjS8OQ==}
     engines: {node: '>=12'}
     cpu: [arm64]
     os: [android]
@@ -1015,8 +1016,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-darwin-64/0.14.54:
-    resolution: {integrity: sha512-jtdKWV3nBviOd5v4hOpkVmpxsBy90CGzebpbO9beiqUYVMBtSc0AL9zGftFuBon7PNDcdvNCEuQqw2x0wP9yug==}
+  /esbuild-darwin-64/0.15.18:
+    resolution: {integrity: sha512-2WAvs95uPnVJPuYKP0Eqx+Dl/jaYseZEUUT1sjg97TJa4oBtbAKnPnl3b5M9l51/nbx7+QAEtuummJZW0sBEmg==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [darwin]
@@ -1024,8 +1025,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-darwin-arm64/0.14.54:
-    resolution: {integrity: sha512-OPafJHD2oUPyvJMrsCvDGkRrVCar5aVyHfWGQzY1dWnzErjrDuSETxwA2HSsyg2jORLY8yBfzc1MIpUkXlctmw==}
+  /esbuild-darwin-arm64/0.15.18:
+    resolution: {integrity: sha512-tKPSxcTJ5OmNb1btVikATJ8NftlyNlc8BVNtyT/UAr62JFOhwHlnoPrhYWz09akBLHI9nElFVfWSTSRsrZiDUA==}
     engines: {node: '>=12'}
     cpu: [arm64]
     os: [darwin]
@@ -1033,8 +1034,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-freebsd-64/0.14.54:
-    resolution: {integrity: sha512-OKwd4gmwHqOTp4mOGZKe/XUlbDJ4Q9TjX0hMPIDBUWWu/kwhBAudJdBoxnjNf9ocIB6GN6CPowYpR/hRCbSYAg==}
+  /esbuild-freebsd-64/0.15.18:
+    resolution: {integrity: sha512-TT3uBUxkteAjR1QbsmvSsjpKjOX6UkCstr8nMr+q7zi3NuZ1oIpa8U41Y8I8dJH2fJgdC3Dj3CXO5biLQpfdZA==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [freebsd]
@@ -1042,8 +1043,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-freebsd-arm64/0.14.54:
-    resolution: {integrity: sha512-sFwueGr7OvIFiQT6WeG0jRLjkjdqWWSrfbVwZp8iMP+8UHEHRBvlaxL6IuKNDwAozNUmbb8nIMXa7oAOARGs1Q==}
+  /esbuild-freebsd-arm64/0.15.18:
+    resolution: {integrity: sha512-R/oVr+X3Tkh+S0+tL41wRMbdWtpWB8hEAMsOXDumSSa6qJR89U0S/PpLXrGF7Wk/JykfpWNokERUpCeHDl47wA==}
     engines: {node: '>=12'}
     cpu: [arm64]
     os: [freebsd]
@@ -1051,8 +1052,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-32/0.14.54:
-    resolution: {integrity: sha512-1ZuY+JDI//WmklKlBgJnglpUL1owm2OX+8E1syCD6UAxcMM/XoWd76OHSjl/0MR0LisSAXDqgjT3uJqT67O3qw==}
+  /esbuild-linux-32/0.15.18:
+    resolution: {integrity: sha512-lphF3HiCSYtaa9p1DtXndiQEeQDKPl9eN/XNoBf2amEghugNuqXNZA/ZovthNE2aa4EN43WroO0B85xVSjYkbg==}
     engines: {node: '>=12'}
     cpu: [ia32]
     os: [linux]
@@ -1060,8 +1061,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-64/0.14.54:
-    resolution: {integrity: sha512-EgjAgH5HwTbtNsTqQOXWApBaPVdDn7XcK+/PtJwZLT1UmpLoznPd8c5CxqsH2dQK3j05YsB3L17T8vE7cp4cCg==}
+  /esbuild-linux-64/0.15.18:
+    resolution: {integrity: sha512-hNSeP97IviD7oxLKFuii5sDPJ+QHeiFTFLoLm7NZQligur8poNOWGIgpQ7Qf8Balb69hptMZzyOBIPtY09GZYw==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [linux]
@@ -1069,8 +1070,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-arm/0.14.54:
-    resolution: {integrity: sha512-qqz/SjemQhVMTnvcLGoLOdFpCYbz4v4fUo+TfsWG+1aOu70/80RV6bgNpR2JCrppV2moUQkww+6bWxXRL9YMGw==}
+  /esbuild-linux-arm/0.15.18:
+    resolution: {integrity: sha512-UH779gstRblS4aoS2qpMl3wjg7U0j+ygu3GjIeTonCcN79ZvpPee12Qun3vcdxX+37O5LFxz39XeW2I9bybMVA==}
     engines: {node: '>=12'}
     cpu: [arm]
     os: [linux]
@@ -1078,8 +1079,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-arm64/0.14.54:
-    resolution: {integrity: sha512-WL71L+0Rwv+Gv/HTmxTEmpv0UgmxYa5ftZILVi2QmZBgX3q7+tDeOQNqGtdXSdsL8TQi1vIaVFHUPDe0O0kdig==}
+  /esbuild-linux-arm64/0.15.18:
+    resolution: {integrity: sha512-54qr8kg/6ilcxd+0V3h9rjT4qmjc0CccMVWrjOEM/pEcUzt8X62HfBSeZfT2ECpM7104mk4yfQXkosY8Quptug==}
     engines: {node: '>=12'}
     cpu: [arm64]
     os: [linux]
@@ -1087,8 +1088,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-mips64le/0.14.54:
-    resolution: {integrity: sha512-qTHGQB8D1etd0u1+sB6p0ikLKRVuCWhYQhAHRPkO+OF3I/iSlTKNNS0Lh2Oc0g0UFGguaFZZiPJdJey3AGpAlw==}
+  /esbuild-linux-mips64le/0.15.18:
+    resolution: {integrity: sha512-Mk6Ppwzzz3YbMl/ZZL2P0q1tnYqh/trYZ1VfNP47C31yT0K8t9s7Z077QrDA/guU60tGNp2GOwCQnp+DYv7bxQ==}
     engines: {node: '>=12'}
     cpu: [mips64el]
     os: [linux]
@@ -1096,8 +1097,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-ppc64le/0.14.54:
-    resolution: {integrity: sha512-j3OMlzHiqwZBDPRCDFKcx595XVfOfOnv68Ax3U4UKZ3MTYQB5Yz3X1mn5GnodEVYzhtZgxEBidLWeIs8FDSfrQ==}
+  /esbuild-linux-ppc64le/0.15.18:
+    resolution: {integrity: sha512-b0XkN4pL9WUulPTa/VKHx2wLCgvIAbgwABGnKMY19WhKZPT+8BxhZdqz6EgkqCLld7X5qiCY2F/bfpUUlnFZ9w==}
     engines: {node: '>=12'}
     cpu: [ppc64]
     os: [linux]
@@ -1105,8 +1106,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-riscv64/0.14.54:
-    resolution: {integrity: sha512-y7Vt7Wl9dkOGZjxQZnDAqqn+XOqFD7IMWiewY5SPlNlzMX39ocPQlOaoxvT4FllA5viyV26/QzHtvTjVNOxHZg==}
+  /esbuild-linux-riscv64/0.15.18:
+    resolution: {integrity: sha512-ba2COaoF5wL6VLZWn04k+ACZjZ6NYniMSQStodFKH/Pu6RxzQqzsmjR1t9QC89VYJxBeyVPTaHuBMCejl3O/xg==}
     engines: {node: '>=12'}
     cpu: [riscv64]
     os: [linux]
@@ -1114,8 +1115,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-linux-s390x/0.14.54:
-    resolution: {integrity: sha512-zaHpW9dziAsi7lRcyV4r8dhfG1qBidQWUXweUjnw+lliChJqQr+6XD71K41oEIC3Mx1KStovEmlzm+MkGZHnHA==}
+  /esbuild-linux-s390x/0.15.18:
+    resolution: {integrity: sha512-VbpGuXEl5FCs1wDVp93O8UIzl3ZrglgnSQ+Hu79g7hZu6te6/YHgVJxCM2SqfIila0J3k0csfnf8VD2W7u2kzQ==}
     engines: {node: '>=12'}
     cpu: [s390x]
     os: [linux]
@@ -1123,8 +1124,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-netbsd-64/0.14.54:
-    resolution: {integrity: sha512-PR01lmIMnfJTgeU9VJTDY9ZerDWVFIUzAtJuDHwwceppW7cQWjBBqP48NdeRtoP04/AtO9a7w3viI+PIDr6d+w==}
+  /esbuild-netbsd-64/0.15.18:
+    resolution: {integrity: sha512-98ukeCdvdX7wr1vUYQzKo4kQ0N2p27H7I11maINv73fVEXt2kyh4K4m9f35U1K43Xc2QGXlzAw0K9yoU7JUjOg==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [netbsd]
@@ -1132,8 +1133,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-openbsd-64/0.14.54:
-    resolution: {integrity: sha512-Qyk7ikT2o7Wu76UsvvDS5q0amJvmRzDyVlL0qf5VLsLchjCa1+IAvd8kTBgUxD7VBUUVgItLkk609ZHUc1oCaw==}
+  /esbuild-openbsd-64/0.15.18:
+    resolution: {integrity: sha512-yK5NCcH31Uae076AyQAXeJzt/vxIo9+omZRKj1pauhk3ITuADzuOx5N2fdHrAKPxN+zH3w96uFKlY7yIn490xQ==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [openbsd]
@@ -1141,8 +1142,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-sunos-64/0.14.54:
-    resolution: {integrity: sha512-28GZ24KmMSeKi5ueWzMcco6EBHStL3B6ubM7M51RmPwXQGLe0teBGJocmWhgwccA1GeFXqxzILIxXpHbl9Q/Kw==}
+  /esbuild-sunos-64/0.15.18:
+    resolution: {integrity: sha512-On22LLFlBeLNj/YF3FT+cXcyKPEI263nflYlAhz5crxtp3yRG1Ugfr7ITyxmCmjm4vbN/dGrb/B7w7U8yJR9yw==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [sunos]
@@ -1150,8 +1151,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-windows-32/0.14.54:
-    resolution: {integrity: sha512-T+rdZW19ql9MjS7pixmZYVObd9G7kcaZo+sETqNH4RCkuuYSuv9AGHUVnPoP9hhuE1WM1ZimHz1CIBHBboLU7w==}
+  /esbuild-windows-32/0.15.18:
+    resolution: {integrity: sha512-o+eyLu2MjVny/nt+E0uPnBxYuJHBvho8vWsC2lV61A7wwTWC3jkN2w36jtA+yv1UgYkHRihPuQsL23hsCYGcOQ==}
     engines: {node: '>=12'}
     cpu: [ia32]
     os: [win32]
@@ -1159,8 +1160,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-windows-64/0.14.54:
-    resolution: {integrity: sha512-AoHTRBUuYwXtZhjXZbA1pGfTo8cJo3vZIcWGLiUcTNgHpJJMC1rVA44ZereBHMJtotyN71S8Qw0npiCIkW96cQ==}
+  /esbuild-windows-64/0.15.18:
+    resolution: {integrity: sha512-qinug1iTTaIIrCorAUjR0fcBk24fjzEedFYhhispP8Oc7SFvs+XeW3YpAKiKp8dRpizl4YYAhxMjlftAMJiaUw==}
     engines: {node: '>=12'}
     cpu: [x64]
     os: [win32]
@@ -1168,8 +1169,8 @@ packages:
     dev: true
     optional: true
 
-  /esbuild-windows-arm64/0.14.54:
-    resolution: {integrity: sha512-M0kuUvXhot1zOISQGXwWn6YtS+Y/1RT9WrVIOywZnJHo3jCDyewAc79aKNQWFCQm+xNHVTq9h8dZKvygoXQQRg==}
+  /esbuild-windows-arm64/0.15.18:
+    resolution: {integrity: sha512-q9bsYzegpZcLziq0zgUi5KqGVtfhjxGbnksaBFYmWLxeV/S1fK4OLdq2DFYnXcLMjlZw2L0jLsk1eGoB522WXQ==}
     engines: {node: '>=12'}
     cpu: [arm64]
     os: [win32]
@@ -1177,33 +1178,34 @@ packages:
     dev: true
     optional: true
 
-  /esbuild/0.14.54:
-    resolution: {integrity: sha512-Cy9llcy8DvET5uznocPyqL3BFRrFXSVqbgpMJ9Wz8oVjZlh/zUSNbPRbov0VX7VxN2JH1Oa0uNxZ7eLRb62pJA==}
+  /esbuild/0.15.18:
+    resolution: {integrity: sha512-x/R72SmW3sSFRm5zrrIjAhCeQSAWoni3CmHEqfQrZIQTM3lVCdehdwuIqaOtfC2slvpdlLa62GYoN8SxT23m6Q==}
     engines: {node: '>=12'}
     hasBin: true
     requiresBuild: true
     optionalDependencies:
-      '@esbuild/linux-loong64': 0.14.54
-      esbuild-android-64: 0.14.54
-      esbuild-android-arm64: 0.14.54
-      esbuild-darwin-64: 0.14.54
-      esbuild-darwin-arm64: 0.14.54
-      esbuild-freebsd-64: 0.14.54
-      esbuild-freebsd-arm64: 0.14.54
-      esbuild-linux-32: 0.14.54
-      esbuild-linux-64: 0.14.54
-      esbuild-linux-arm: 0.14.54
-      esbuild-linux-arm64: 0.14.54
-      esbuild-linux-mips64le: 0.14.54
-      esbuild-linux-ppc64le: 0.14.54
-      esbuild-linux-riscv64: 0.14.54
-      esbuild-linux-s390x: 0.14.54
-      esbuild-netbsd-64: 0.14.54
-      esbuild-openbsd-64: 0.14.54
-      esbuild-sunos-64: 0.14.54
-      esbuild-windows-32: 0.14.54
-      esbuild-windows-64: 0.14.54
-      esbuild-windows-arm64: 0.14.54
+      '@esbuild/android-arm': 0.15.18
+      '@esbuild/linux-loong64': 0.15.18
+      esbuild-android-64: 0.15.18
+      esbuild-android-arm64: 0.15.18
+      esbuild-darwin-64: 0.15.18
+      esbuild-darwin-arm64: 0.15.18
+      esbuild-freebsd-64: 0.15.18
+      esbuild-freebsd-arm64: 0.15.18
+      esbuild-linux-32: 0.15.18
+      esbuild-linux-64: 0.15.18
+      esbuild-linux-arm: 0.15.18
+      esbuild-linux-arm64: 0.15.18
+      esbuild-linux-mips64le: 0.15.18
+      esbuild-linux-ppc64le: 0.15.18
+      esbuild-linux-riscv64: 0.15.18
+      esbuild-linux-s390x: 0.15.18
+      esbuild-netbsd-64: 0.15.18
+      esbuild-openbsd-64: 0.15.18
+      esbuild-sunos-64: 0.15.18
+      esbuild-windows-32: 0.15.18
+      esbuild-windows-64: 0.15.18
+      esbuild-windows-arm64: 0.15.18
     dev: true
 
   /escalade/3.1.1:
@@ -1233,16 +1235,16 @@ packages:
       source-map: 0.6.1
     dev: true
 
-  /eslint-config-prettier/8.5.0_eslint@8.22.0:
+  /eslint-config-prettier/8.5.0_eslint@8.29.0:
     resolution: {integrity: sha512-obmWKLUNCnhtQRKc+tmnYuQl0pFU1ibYJQ5BGhTVB08bHe9wC8qUeG7c08dj9XX+AuPj1YSGSQIHl1pnDHZR0Q==}
     hasBin: true
     peerDependencies:
       eslint: '>=7.0.0'
     dependencies:
-      eslint: 8.22.0
+      eslint: 8.29.0
     dev: true
 
-  /eslint-plugin-prettier/4.2.1_2xd4q2tc5cqa5as7uugqhp6oue:
+  /eslint-plugin-prettier/4.2.1_nrhoyyjffvfyk4vtlt5destxgm:
     resolution: {integrity: sha512-f/0rXLXUt0oFYs8ra4w49wYZBG5GKZpAYsJSm6rnYL5uVDjd+zowwMwVZHnAjf4edNrKpCDYfXDgmRE/Ak7QyQ==}
     engines: {node: '>=12.0.0'}
     peerDependencies:
@@ -1253,12 +1255,13 @@ packages:
       eslint-config-prettier:
         optional: true
     dependencies:
-      eslint: 8.22.0
-      prettier: 2.7.1
+      eslint: 8.29.0
+      eslint-config-prettier: 8.5.0_eslint@8.29.0
+      prettier: 2.8.0
       prettier-linter-helpers: 1.0.0
     dev: true
 
-  /eslint-plugin-prettier/4.2.1_i2cojdczqdiurzgttlwdgf764e:
+  /eslint-plugin-prettier/4.2.1_yz7jvh5l5mgl5xhpwlgtkgcccy:
     resolution: {integrity: sha512-f/0rXLXUt0oFYs8ra4w49wYZBG5GKZpAYsJSm6rnYL5uVDjd+zowwMwVZHnAjf4edNrKpCDYfXDgmRE/Ak7QyQ==}
     engines: {node: '>=12.0.0'}
     peerDependencies:
@@ -1269,25 +1272,24 @@ packages:
       eslint-config-prettier:
         optional: true
     dependencies:
-      eslint: 8.22.0
-      eslint-config-prettier: 8.5.0_eslint@8.22.0
-      prettier: 2.7.1
+      eslint: 8.29.0
+      prettier: 2.8.0
       prettier-linter-helpers: 1.0.0
     dev: true
 
-  /eslint-plugin-vue/9.3.0_eslint@8.22.0:
-    resolution: {integrity: sha512-iscKKkBZgm6fGZwFt6poRoWC0Wy2dQOlwUPW++CiPoQiw1enctV2Hj5DBzzjJZfyqs+FAXhgzL4q0Ww03AgSmQ==}
+  /eslint-plugin-vue/9.8.0_eslint@8.29.0:
+    resolution: {integrity: sha512-E/AXwcTzunyzM83C2QqDHxepMzvI2y6x+mmeYHbVDQlKFqmKYvRrhaVixEeeG27uI44p9oKDFiyCRw4XxgtfHA==}
     engines: {node: ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: ^6.2.0 || ^7.0.0 || ^8.0.0
     dependencies:
-      eslint: 8.22.0
-      eslint-utils: 3.0.0_eslint@8.22.0
+      eslint: 8.29.0
+      eslint-utils: 3.0.0_eslint@8.29.0
       natural-compare: 1.4.0
       nth-check: 2.1.1
-      postcss-selector-parser: 6.0.10
-      semver: 7.3.7
-      vue-eslint-parser: 9.0.3_eslint@8.22.0
+      postcss-selector-parser: 6.0.11
+      semver: 7.3.8
+      vue-eslint-parser: 9.1.0_eslint@8.29.0
       xml-name-validator: 4.0.0
     transitivePeerDependencies:
       - supports-color
@@ -1309,13 +1311,13 @@ packages:
       estraverse: 5.3.0
     dev: true
 
-  /eslint-utils/3.0.0_eslint@8.22.0:
+  /eslint-utils/3.0.0_eslint@8.29.0:
     resolution: {integrity: sha512-uuQC43IGctw68pJA1RgbQS8/NP7rch6Cwd4j3ZBtgo4/8Flj4eGE7ZYSZRN3iq5pVUv6GPdW5Z1RFleo84uLDA==}
     engines: {node: ^10.0.0 || ^12.0.0 || >= 14.0.0}
     peerDependencies:
       eslint: '>=5'
     dependencies:
-      eslint: 8.22.0
+      eslint: 8.29.0
       eslint-visitor-keys: 2.1.0
     dev: true
 
@@ -1329,14 +1331,15 @@ packages:
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dev: true
 
-  /eslint/8.22.0:
-    resolution: {integrity: sha512-ci4t0sz6vSRKdmkOGmprBo6fmI4PrphDFMy5JEq/fNS0gQkJM3rLmrqcp8ipMcdobH3KtUP40KniAE9W19S4wA==}
+  /eslint/8.29.0:
+    resolution: {integrity: sha512-isQ4EEiyUjZFbEKvEGJKKGBwXtvXX+zJbkVKCgTuB9t/+jUBcy8avhkEwWJecI15BkRkOYmvIM5ynbhRjEkoeg==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     hasBin: true
     dependencies:
-      '@eslint/eslintrc': 1.3.0
-      '@humanwhocodes/config-array': 0.10.4
-      '@humanwhocodes/gitignore-to-minimatch': 1.0.2
+      '@eslint/eslintrc': 1.3.3
+      '@humanwhocodes/config-array': 0.11.7
+      '@humanwhocodes/module-importer': 1.0.1
+      '@nodelib/fs.walk': 1.2.8
       ajv: 6.12.6
       chalk: 4.1.2
       cross-spawn: 7.0.3
@@ -1344,23 +1347,23 @@ packages:
       doctrine: 3.0.0
       escape-string-regexp: 4.0.0
       eslint-scope: 7.1.1
-      eslint-utils: 3.0.0_eslint@8.22.0
+      eslint-utils: 3.0.0_eslint@8.29.0
       eslint-visitor-keys: 3.3.0
-      espree: 9.3.3
+      espree: 9.4.1
       esquery: 1.4.0
       esutils: 2.0.3
       fast-deep-equal: 3.1.3
       file-entry-cache: 6.0.1
       find-up: 5.0.0
-      functional-red-black-tree: 1.0.1
       glob-parent: 6.0.2
-      globals: 13.17.0
-      globby: 11.1.0
+      globals: 13.18.0
       grapheme-splitter: 1.0.4
-      ignore: 5.2.0
+      ignore: 5.2.1
       import-fresh: 3.3.0
       imurmurhash: 0.1.4
       is-glob: 4.0.3
+      is-path-inside: 3.0.3
+      js-sdsl: 4.2.0
       js-yaml: 4.1.0
       json-stable-stringify-without-jsonify: 1.0.1
       levn: 0.4.1
@@ -1372,17 +1375,16 @@ packages:
       strip-ansi: 6.0.1
       strip-json-comments: 3.1.1
       text-table: 0.2.0
-      v8-compile-cache: 2.3.0
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /espree/9.3.3:
-    resolution: {integrity: sha512-ORs1Rt/uQTqUKjDdGCyrtYxbazf5umATSf/K4qxjmZHORR6HJk+2s/2Pqe+Kk49HHINC/xNIrGfgh8sZcll0ng==}
+  /espree/9.4.1:
+    resolution: {integrity: sha512-XwctdmTO6SIvCzd9810yyNzIrOrqNYV9Koizx4C/mRhf9uq0o4yHoCEU/670pOxOL/MSraektvSAji79kX90Vg==}
     engines: {node: ^12.22.0 || ^14.17.0 || >=16.0.0}
     dependencies:
-      acorn: 8.8.0
-      acorn-jsx: 5.3.2_acorn@8.8.0
+      acorn: 8.8.1
+      acorn-jsx: 5.3.2_acorn@8.8.1
       eslint-visitor-keys: 3.3.0
     dev: true
 
@@ -1432,8 +1434,8 @@ packages:
     resolution: {integrity: sha512-xJuoT5+L99XlZ8twedaRf6Ax2TgQVxvgZOYoPKqZufmJib0tL2tegPBOZb1pVNgIhlqDlA0eO0c3wBvQcmzx4w==}
     dev: true
 
-  /fast-glob/3.2.11:
-    resolution: {integrity: sha512-xrO3+1bxSo3ZVHAnqzyuewYT6aMFHRAd4Kcs92MAonjwQZLsK9d0SF1IyQ3k5PoirxTW0Oe/RqFgMQ6TcNE5Ew==}
+  /fast-glob/3.2.12:
+    resolution: {integrity: sha512-DVj4CQIYYow0BlaelwK1pHl5n5cRSJfM60UA0zK891sVInoPri2Ekj7+e1CT3/3qxXenpI+nBBmQAcJPJgaj4w==}
     engines: {node: '>=8.6.0'}
     dependencies:
       '@nodelib/fs.stat': 2.0.5
@@ -1451,8 +1453,8 @@ packages:
     resolution: {integrity: sha512-DCXu6Ifhqcks7TZKY3Hxp3y6qphY5SJZmrWMDrKcERSOXWQdMhU9Ig/PYrzyw/ul9jOIyh0N4M0tbC5hodg8dw==}
     dev: true
 
-  /fastq/1.13.0:
-    resolution: {integrity: sha512-YpkpUnK8od0o1hmeSc7UUs/eB/vIPWJYjKck2QKIzAf71Vm1AAQ3EbuZB3g2JIy+pg+ERD0vqI79KyZiB2e2Nw==}
+  /fastq/1.14.0:
+    resolution: {integrity: sha512-eR2D+V9/ExcbF9ls441yIuN6TI2ED1Y2ZcA5BmMtJsOkWOFRJQ0Jt0g1UwqXJJVAb+V+umH5Dfr8oh4EVP7VVg==}
     dependencies:
       reusify: 1.0.4
     dev: true
@@ -1483,16 +1485,16 @@ packages:
     resolution: {integrity: sha512-dm9s5Pw7Jc0GvMYbshN6zchCA9RgQlzzEZX3vylR9IqFfS8XciblUXOKfW6SiuJ0e13eDYZoZV5wdrev7P3Nwg==}
     engines: {node: ^10.12.0 || >=12.0.0}
     dependencies:
-      flatted: 3.2.6
+      flatted: 3.2.7
       rimraf: 3.0.2
     dev: true
 
-  /flatted/3.2.6:
-    resolution: {integrity: sha512-0sQoMh9s0BYsm+12Huy/rkKxVu4R1+r96YX5cG44rHV0pQ6iC3Q+mkoMFaGWObMFYQxCVT+ssG1ksneA2MI9KQ==}
+  /flatted/3.2.7:
+    resolution: {integrity: sha512-5nqDSxl8nn5BSNxyR3n4I6eDmbolI6WT+QqR547RwxQapgjQBmtktdP+HTBb/a/zLsbzERTONyUB5pefh5TtjQ==}
     dev: true
 
-  /follow-redirects/1.15.1:
-    resolution: {integrity: sha512-yLAMQs+k0b2m7cVxpS1VKJVvoz7SS9Td1zss3XRwXj+ZDH00RJgnuLx7E44wx02kQLrdM3aOOy+FpzS7+8OizA==}
+  /follow-redirects/1.15.2:
+    resolution: {integrity: sha512-VQLG33o04KaQ8uYi2tVNbdrWp1QWxNNea+nmIB4EVM28v0hmP17z7aG1+wAkNzVq4KeXTq3221ye5qTJP91JwA==}
     engines: {node: '>=4.0'}
     peerDependencies:
       debug: '*'
@@ -1533,10 +1535,6 @@ packages:
     resolution: {integrity: sha512-yIovAzMX49sF8Yl58fSCWJ5svSLuaibPxXQJFLmBObTuCr0Mf1KiPopGM9NiFjiYBCbfaa2Fh6breQ6ANVTI0A==}
     dev: true
 
-  /functional-red-black-tree/1.0.1:
-    resolution: {integrity: sha512-dsKNQNdj6xA3T+QlADDA7mOSlX0qiMINjn0cgr+eGHGsbSHzTabcIogz2+p/iqP1Xs6EP/sS2SbqH+brGTbq0g==}
-    dev: true
-
   /get-caller-file/2.0.5:
     resolution: {integrity: sha512-DyFP3BM/3YHTQOCUL/w0OZHR0lpKeGrxotcHWcqNEdnltqFwXVfhEBQ94eIo34AfQpo0rGki4cyIiftY06h2Fg==}
     engines: {node: 6.* || 8.* || >= 10.*}
@@ -1571,8 +1569,8 @@ packages:
       path-is-absolute: 1.0.1
     dev: true
 
-  /globals/13.17.0:
-    resolution: {integrity: sha512-1C+6nQRb1GwGMKm2dH/E7enFAMxGTmGI7/dEdhy/DNelv85w9B72t3uc5frtMNXIbzrarJJ/lTCjcaZwbLJmyw==}
+  /globals/13.18.0:
+    resolution: {integrity: sha512-/mR4KI8Ps2spmoc0Ulu9L7agOF0du1CZNQ3dke8yItYlyKNmGrkONemBbd6V8UTc1Wgcqn21t3WYB7dbRmh6/A==}
     engines: {node: '>=8'}
     dependencies:
       type-fest: 0.20.2
@@ -1584,8 +1582,8 @@ packages:
     dependencies:
       array-union: 2.1.0
       dir-glob: 3.0.1
-      fast-glob: 3.2.11
-      ignore: 5.2.0
+      fast-glob: 3.2.12
+      ignore: 5.2.1
       merge2: 1.4.1
       slash: 3.0.0
     dev: true
@@ -1645,8 +1643,8 @@ packages:
       safer-buffer: 2.1.2
     dev: true
 
-  /ignore/5.2.0:
-    resolution: {integrity: sha512-CmxgYGiEPCLhfLnpPp1MoRmifwEIOgjcHXxOBjv7mY96c+eWScsOP9c112ZyLdWHi0FxHjI+4uVhKYp/gcdRmQ==}
+  /ignore/5.2.1:
+    resolution: {integrity: sha512-d2qQLzTJ9WxQftPAuEQpSPmKqzxePjzVbpAVv62AQ64NTL+wR4JkrVqR/LqFsFEUsHDAiId52mJteHDFuDkElA==}
     engines: {node: '>= 4'}
     dev: true
 
@@ -1685,8 +1683,8 @@ packages:
       binary-extensions: 2.2.0
     dev: true
 
-  /is-core-module/2.10.0:
-    resolution: {integrity: sha512-Erxj2n/LDAZ7H8WNJXd9tw38GYM3dv8rk8Zcs+jJuxYTW7sozH+SS8NtrSjVL1/vpLvWi1hxy96IzjJ3EHTJJg==}
+  /is-core-module/2.11.0:
+    resolution: {integrity: sha512-RRjxlvLDkD1YJwDbroBHMb+cukurkDWNyHx7D3oNB5x9rb5ogcksMC5wHCadcXoo67gVr/+3GFySh3134zi6rw==}
     dependencies:
       has: 1.0.3
     dev: true
@@ -1713,6 +1711,11 @@ packages:
     engines: {node: '>=0.12.0'}
     dev: true
 
+  /is-path-inside/3.0.3:
+    resolution: {integrity: sha512-Fd4gABb+ycGAmKou8eMftCupSir5lRxqf4aD/vd0cD2qc4HL07OjCeuHMr8Ro4CoMaeCKDB0/ECBOVWjTwUvPQ==}
+    engines: {node: '>=8'}
+    dev: true
+
   /is-potential-custom-element-name/1.0.1:
     resolution: {integrity: sha512-bCYeRA2rVibKZd+s2625gGnGF/t7DSqDs4dP7CrLA1m7jKWz6pps0LpYLJN8Q64HtmPKJ1hrN3nzPNKFEKOUiQ==}
     dev: true
@@ -1743,6 +1746,10 @@ packages:
       istanbul-lib-report: 3.0.0
     dev: true
 
+  /js-sdsl/4.2.0:
+    resolution: {integrity: sha512-dyBIzQBDkCqCu+0upx25Y2jGdbTGxE9fshMsCdK0ViOongpV+n5tXRcZY9v7CaVQ79AGS9KA1KHtojxiM7aXSQ==}
+    dev: true
+
   /js-yaml/4.1.0:
     resolution: {integrity: sha512-wpxZs9NoxZaJESJGIZTyDEaYpl0FKSA+FB9aJiyemKhMwkxQg63h4T1KJgUGHpTqPDNRcmmYLugrRjJlBtWvRA==}
     hasBin: true
@@ -1750,8 +1757,8 @@ packages:
       argparse: 2.0.1
     dev: true
 
-  /jsdom/20.0.0:
-    resolution: {integrity: sha512-x4a6CKCgx00uCmP+QakBDFXwjAJ69IkkIWHmtmjd3wvXPcdOS44hfX2vqkOQrVrq8l9DhNNADZRXaCEWvgXtVA==}
+  /jsdom/20.0.3:
+    resolution: {integrity: sha512-SYhBvTh89tTfCD/CRdSOm13mOBa42iTaTyfyEWBdKcGdPxPtLFBXuHR8XHb33YNYaP+lLbmSvBTsnoesCNJEsQ==}
     engines: {node: '>=14'}
     peerDependencies:
       canvas: ^2.5.0
@@ -1760,12 +1767,12 @@ packages:
         optional: true
     dependencies:
       abab: 2.0.6
-      acorn: 8.8.0
-      acorn-globals: 6.0.0
+      acorn: 8.8.1
+      acorn-globals: 7.0.1
       cssom: 0.5.0
       cssstyle: 2.3.0
       data-urls: 3.0.2
-      decimal.js: 10.4.0
+      decimal.js: 10.4.3
       domexception: 4.0.0
       escodegen: 2.0.0
       form-data: 4.0.0
@@ -1773,18 +1780,17 @@ packages:
       http-proxy-agent: 5.0.0
       https-proxy-agent: 5.0.1
       is-potential-custom-element-name: 1.0.1
-      nwsapi: 2.2.1
-      parse5: 7.0.0
+      nwsapi: 2.2.2
+      parse5: 7.1.2
       saxes: 6.0.0
       symbol-tree: 3.2.4
-      tough-cookie: 4.0.0
-      w3c-hr-time: 1.0.2
-      w3c-xmlserializer: 3.0.0
+      tough-cookie: 4.1.2
+      w3c-xmlserializer: 4.0.0
       webidl-conversions: 7.0.0
       whatwg-encoding: 2.0.0
       whatwg-mimetype: 3.0.0
       whatwg-url: 11.0.0
-      ws: 8.8.1
+      ws: 8.11.0
       xml-name-validator: 4.0.0
     transitivePeerDependencies:
       - bufferutil
@@ -1832,8 +1838,8 @@ packages:
     resolution: {integrity: sha512-mKnC+QJ9pWVzv+C4/U3rRsHapFfHvQFoFB92e52xeyGMcX6/OlIl78je1u8vePzYZSkkogMPJ2yjxxsb89cxyw==}
     dev: false
 
-  /lodash-unified/1.0.2_3ib2ivapxullxkx3xftsimdk7u:
-    resolution: {integrity: sha512-OGbEy+1P+UT26CYi4opY4gebD8cWRDxAT6MAObIVQMiqYdxZr1g3QHWCToVsm31x2NkLS4K3+MC2qInaRMa39g==}
+  /lodash-unified/1.0.3_3ib2ivapxullxkx3xftsimdk7u:
+    resolution: {integrity: sha512-WK9qSozxXOD7ZJQlpSqOT+om2ZfcT4yO+03FuzAHD0wF6S0l0090LRPDx3vhTTLZ8cFKpBn+IOcVXK6qOcIlfQ==}
     peerDependencies:
       '@types/lodash-es': '*'
       lodash: '*'
@@ -1851,8 +1857,8 @@ packages:
   /lodash/4.17.21:
     resolution: {integrity: sha512-v2kDEe57lecTulaDIuNTPy3Ry4gLGJ6Z1O3vE1krgXZNrsQ+LFTGHVxVjcXPs17LhbZVGedAJv8XZ1tvj5FvSg==}
 
-  /loupe/2.3.4:
-    resolution: {integrity: sha512-OvKfgCC2Ndby6aSTREl5aCCPTNIzlDfQZvZxNUrBrihDhL3xcrYegTblhmEiCrg2kKQz4XsFIaemE5BF4ybSaQ==}
+  /loupe/2.3.6:
+    resolution: {integrity: sha512-RaPMZKiMy8/JruncMU5Bt6na1eftNoo++R4Y+N2FrxkDVTrGvcyzFTsaGif4QTeKESheMGegbhw6iUAq+5A8zA==}
     dependencies:
       get-func-name: 2.0.0
     dev: true
@@ -1918,6 +1924,10 @@ packages:
     engines: {node: ^10 || ^12 || ^13.7 || ^14 || >=15.0.1}
     hasBin: true
 
+  /natural-compare-lite/1.4.0:
+    resolution: {integrity: sha512-Tj+HTDSJJKaZnfiuw+iaF9skdPpTo2GtEly5JHnWV/hfv2Qj/9RKsGISQtLh2ox3l5EAGw487hnBee0sIJ6v2g==}
+    dev: true
+
   /natural-compare/1.4.0:
     resolution: {integrity: sha512-OWND8ei3VtNC9h7V60qff3SVobHr996CTwgxubgyQYEpg290h9J0buyECNNJexkFm5sOajh5G116RYA1c8ZMSw==}
     dev: true
@@ -1937,8 +1947,8 @@ packages:
       boolbase: 1.0.0
     dev: true
 
-  /nwsapi/2.2.1:
-    resolution: {integrity: sha512-JYOWTeFoS0Z93587vRJgASD5Ut11fYl5NyihP3KrYBvMe1FRRs6RN7m20SA/16GM4P6hTnZjT+UmDOt38UeXNg==}
+  /nwsapi/2.2.2:
+    resolution: {integrity: sha512-90yv+6538zuvUMnN+zCr8LuV6bPFdq50304114vJYJ8RDyK8D5O9Phpbd6SZWgI7PwzmmfN1upeOJlvybDSgCw==}
     dev: true
 
   /once/1.4.0:
@@ -1992,10 +2002,10 @@ packages:
       callsites: 3.1.0
     dev: true
 
-  /parse5/7.0.0:
-    resolution: {integrity: sha512-y/t8IXSPWTuRZqXc0ajH/UwDj4mnqLEbSttNbThcFhGrZuOyoyvNBO85PBp2jQa55wY9d07PBNjsK8ZP3K5U6g==}
+  /parse5/7.1.2:
+    resolution: {integrity: sha512-Czj1WaSVpaoj0wbhMzLmWD69anp2WH7FXMB9n1Sy8/ZFF9jolSQVMu1Ij5WIyGmcBmhk7EOndpO4mIpihVqAXw==}
     dependencies:
-      entities: 4.3.1
+      entities: 4.4.0
     dev: true
 
   /path-exists/4.0.0:
@@ -2034,19 +2044,19 @@ packages:
     engines: {node: '>=8.6'}
     dev: true
 
-  /pinia-plugin-persistedstate/2.1.1_pinia@2.0.18:
-    resolution: {integrity: sha512-HUgsU5IRtM75eAQiIqzT3p1oPEuYH1/B2ipTMU++yE+FV0LkHaBswdKXs0RMWYCmugO8s62oxLTh/N1dLNp+5A==}
+  /pinia-plugin-persistedstate/2.4.0_pinia@2.0.27:
+    resolution: {integrity: sha512-bQcpv47jk3ISl+InuJWsFaS/K7pRZ97kfoD2WCf/suhnlLy48k3BnFM2tI6YZ1xMsDaPv4yOsaPuPAUuSmEO2Q==}
     peerDependencies:
       pinia: ^2.0.0
     peerDependenciesMeta:
       pinia:
         optional: true
     dependencies:
-      pinia: 2.0.18_j6bzmzd4ujpabbp5objtwxyjp4
+      pinia: 2.0.27_mgnvym7yiazkylwwogi5r767ue
     dev: false
 
-  /pinia/2.0.18_j6bzmzd4ujpabbp5objtwxyjp4:
-    resolution: {integrity: sha512-I5MW05UVX6a5Djka136oH3VzYFiZUgeOApBwFjMx6pL91eHtGVlE3adjNUKLgtwGnrxiBRuJ8+4R3LKJKwnyZg==}
+  /pinia/2.0.27_mgnvym7yiazkylwwogi5r767ue:
+    resolution: {integrity: sha512-nOnXP0OFeL8R4WjAHsterU+11vptda643gH02xKNtSCDPiRzVfRYodOLihLDoa0gL1KKuQKV+KOzEgdt3YvqEw==}
     peerDependencies:
       '@vue/composition-api': ^1.4.0
       typescript: '>=4.4.4'
@@ -2057,22 +2067,22 @@ packages:
       typescript:
         optional: true
     dependencies:
-      '@vue/devtools-api': 6.2.1
-      typescript: 4.7.4
-      vue: 3.2.37
-      vue-demi: 0.13.8_vue@3.2.37
+      '@vue/devtools-api': 6.4.5
+      typescript: 4.9.3
+      vue: 3.2.45
+      vue-demi: 0.13.11_vue@3.2.45
     dev: false
 
-  /postcss-selector-parser/6.0.10:
-    resolution: {integrity: sha512-IQ7TZdoaqbT+LCpShg46jnZVlhWD2w6iQYAcYXfHARZ7X1t/UGhhceQDs5X0cGqKvYlHNOuv7Oa1xmb0oQuA3w==}
+  /postcss-selector-parser/6.0.11:
+    resolution: {integrity: sha512-zbARubNdogI9j7WY4nQJBiNqQf3sLS3wCP4WfOidu+p28LofJqDH1tcXypGrcmMHhDk2t9wGhCsYe/+szLTy1g==}
     engines: {node: '>=4'}
     dependencies:
       cssesc: 3.0.0
       util-deprecate: 1.0.2
     dev: true
 
-  /postcss/8.4.16:
-    resolution: {integrity: sha512-ipHE1XBvKzm5xI7hiHCZJCSugxvsdq2mPnsq5+UF+VHCjiBvtDrlxJfMBToWaP9D5XlgNmcFGqoHmUn0EYEaRQ==}
+  /postcss/8.4.19:
+    resolution: {integrity: sha512-h+pbPsyhlYj6N2ozBmHhHrs9DzGmbaarbLvWipMRO7RLS+v4onj26MPFXA5OBYFxyqYhUJK456SwDcY9H2/zsA==}
     engines: {node: ^10 || ^12 || >=14}
     dependencies:
       nanoid: 3.3.4
@@ -2096,8 +2106,8 @@ packages:
       fast-diff: 1.2.0
     dev: true
 
-  /prettier/2.7.1:
-    resolution: {integrity: sha512-ujppO+MkdPqoVINuDFDRLClm7D78qbDt0/NR+wp5FqEZOoTNAjPHWj17QRhu7geIHJfcNhRk1XVQmF8Bp3ye+g==}
+  /prettier/2.8.0:
+    resolution: {integrity: sha512-9Lmg8hTFZKG0Asr/kW9Bp8tJjRVluO8EJQVfY2T7FMw9T5jy4I/Uvx0Rca/XWf50QQ1/SS48+6IJWnrb+2yemA==}
     engines: {node: '>=10.13.0'}
     hasBin: true
     dev: true
@@ -2111,6 +2121,10 @@ packages:
     engines: {node: '>=6'}
     dev: true
 
+  /querystringify/2.2.0:
+    resolution: {integrity: sha512-FIqgj2EUvTa7R50u0rGsyTftzjYmv/a3hO345bZNrqabNqjtgiDMgmo4mkUjd+nzU5oF3dClKqFIPUKybUyqoQ==}
+    dev: true
+
   /queue-microtask/1.2.3:
     resolution: {integrity: sha512-NuaNSa6flKT5JaSYQzJok04JzTL1CA6aGhv5rfLW3PgqA+M2ChpZQnAC8h8i4ZFkBS8X5RqkDBHA7r4hej3K9A==}
     dev: true
@@ -2132,6 +2146,10 @@ packages:
     engines: {node: '>=0.10.0'}
     dev: true
 
+  /requires-port/1.0.0:
+    resolution: {integrity: sha512-KigOCHcocU3XODJxsu8i/j8T9tzT4adHiecwORRQ0ZZFcp7ahwXuRU1m+yuO90C5ZUyGeGfocHDI14M3L3yDAQ==}
+    dev: true
+
   /resolve-from/4.0.0:
     resolution: {integrity: sha512-pb/MYmXstAkysRFx8piNI1tGFNQIFA3vkE3Gq4EuA1dF6gHp/+vgZqsCGJapvy8N3Q+4o7FwvquPJcnZ7RYy4g==}
     engines: {node: '>=4'}
@@ -2141,7 +2159,7 @@ packages:
     resolution: {integrity: sha512-nBpuuYuY5jFsli/JIs1oldw6fOQCBioohqWZg/2hiaOybXOft4lonv85uDOKXdf8rhyK159cxU5cDcK/NKk8zw==}
     hasBin: true
     dependencies:
-      is-core-module: 2.10.0
+      is-core-module: 2.11.0
       path-parse: 1.0.7
       supports-preserve-symlinks-flag: 1.0.0
     dev: true
@@ -2158,8 +2176,8 @@ packages:
       glob: 7.2.3
     dev: true
 
-  /rollup/2.77.3:
-    resolution: {integrity: sha512-/qxNTG7FbmefJWoeeYJFbHehJ2HNWnjkAFRKzWN/45eNBBF/r8lo992CwcJXEzyVxs5FmfId+vTSTQDb+bxA+g==}
+  /rollup/2.79.1:
+    resolution: {integrity: sha512-uKxbd0IhMZOhjAiD5oAFp7BqvkA4Dv47qpOCtaNvng4HBwdbWtdOh8f5nZNuk2rp51PMGk3bzfWu5oayNEuYnw==}
     engines: {node: '>=10.0.0'}
     hasBin: true
     optionalDependencies:
@@ -2172,16 +2190,12 @@ packages:
       queue-microtask: 1.2.3
     dev: true
 
-  /safe-buffer/5.1.2:
-    resolution: {integrity: sha512-Gd2UZBJDkXlY7GbJxfsE8/nvKkUEU1G38c1siN6QP6a9PT9MmHB8GnpscSmMJSoF8LOIrt8ud/wPtojys4G6+g==}
-    dev: true
-
   /safer-buffer/2.1.2:
     resolution: {integrity: sha512-YZo3K82SD7Riyi0E1EQPojLz7kpepnSQI9IyPbHHg1XXXevb5dJI7tpyN2ADxGcQbHG7vcyRHk0cbwqcQriUtg==}
     dev: true
 
-  /sass/1.54.4:
-    resolution: {integrity: sha512-3tmF16yvnBwtlPrNBHw/H907j8MlOX8aTBnlNX1yrKx24RKcJGPyLhFUwkoKBKesR3unP93/2z14Ll8NicwQUA==}
+  /sass/1.56.1:
+    resolution: {integrity: sha512-VpEyKpyBPCxE7qGDtOcdJ6fFbcpOM+Emu7uZLxVrkX8KVU/Dp5UF7WLvzqRuUhB6mqqQt1xffLoG+AndxTZrCQ==}
     engines: {node: '>=12.0.0'}
     hasBin: true
     dependencies:
@@ -2202,8 +2216,8 @@ packages:
     hasBin: true
     dev: true
 
-  /semver/7.3.7:
-    resolution: {integrity: sha512-QlYTucUYOews+WeEujDoEGziz4K6c47V/Bd+LjSSYcA94p+DmINdf7ncaUinThfvZyu13lN9OY1XDxt8C0Tw0g==}
+  /semver/7.3.8:
+    resolution: {integrity: sha512-NB1ctGL5rlHrPJtFDVIVzTyQylMLu9N9VICA6HSFJo8MCGVTMW6gfpicwKmmK/dAjTOrqu5l63JJOpDSrAis3A==}
     engines: {node: '>=10'}
     hasBin: true
     dependencies:
@@ -2241,6 +2255,7 @@ packages:
 
   /sourcemap-codec/1.4.8:
     resolution: {integrity: sha512-9NykojV5Uih4lgo5So5dtw+f0JgJX30KCNI8gwhz2J9A15wD0Ml6tjHKwf6fTSa6fAdVBdZeNOs9eJ71qCk8vA==}
+    deprecated: Please use @jridgewell/sourcemap-codec instead
 
   /string-width/4.2.3:
     resolution: {integrity: sha512-wKyQRQpjJ0sIp62ErSZdGsjMJWsap5oRNihHhu6G7JVO/9jIB6UyevL+tXuOqrng8j/cxKTWyWUwvSTriiZz/g==}
@@ -2297,8 +2312,8 @@ packages:
     engines: {node: '>=14.0.0'}
     dev: true
 
-  /tinyspy/1.0.0:
-    resolution: {integrity: sha512-FI5B2QdODQYDRjfuLF+OrJ8bjWRMCXokQPcwKm0W3IzcbUmBNv536cQc7eXGoAuXphZwgx1DFbqImwzz08Fnhw==}
+  /tinyspy/1.0.2:
+    resolution: {integrity: sha512-bSGlgwLBYf7PnUsQ6WOc6SJ3pGOcd+d8AA6EUnLDDM0kWEstC1JIlSZA3UNliDXhd9ABoS7hiRBDCu+XP/sf1Q==}
     engines: {node: '>=14.0.0'}
     dev: true
 
@@ -2313,13 +2328,14 @@ packages:
       is-number: 7.0.0
     dev: true
 
-  /tough-cookie/4.0.0:
-    resolution: {integrity: sha512-tHdtEpQCMrc1YLrMaqXXcj6AxhYi/xgit6mZu1+EDWUn+qhUf8wMQoFIy9NXuq23zAwtcB0t/MjACGR18pcRbg==}
+  /tough-cookie/4.1.2:
+    resolution: {integrity: sha512-G9fqXWoYFZgTc2z8Q5zaHy/vJMjm+WV0AkAeHxVCQiEB1b+dGvWzFW6QV07cY5jQ5gRkeid2qIkzkxUnmoQZUQ==}
     engines: {node: '>=6'}
     dependencies:
       psl: 1.9.0
       punycode: 2.1.1
-      universalify: 0.1.2
+      universalify: 0.2.0
+      url-parse: 1.5.10
     dev: true
 
   /tr46/3.0.0:
@@ -2333,14 +2349,14 @@ packages:
     resolution: {integrity: sha512-Xni35NKzjgMrwevysHTCArtLDpPvye8zV/0E4EyYn43P7/7qvQwPh9BGkHewbMulVntbigmcT7rdX3BNo9wRJg==}
     dev: true
 
-  /tsutils/3.21.0_typescript@4.7.4:
+  /tsutils/3.21.0_typescript@4.9.3:
     resolution: {integrity: sha512-mHKK3iUXL+3UF6xL5k0PEhKRUBKPBCv/+RkEOpjRWxxx27KKRBmmA60A9pgOUvMi8GKhRMPEmjBRPzs2W7O1OA==}
     engines: {node: '>= 6'}
     peerDependencies:
       typescript: '>=2.8.0 || >= 3.2.0-dev || >= 3.3.0-dev || >= 3.4.0-dev || >= 3.5.0-dev || >= 3.6.0-dev || >= 3.6.0-beta || >= 3.7.0-dev || >= 3.7.0-beta'
     dependencies:
       tslib: 1.14.1
-      typescript: 4.7.4
+      typescript: 4.9.3
     dev: true
 
   /type-check/0.3.2:
@@ -2367,13 +2383,13 @@ packages:
     engines: {node: '>=10'}
     dev: true
 
-  /typescript/4.7.4:
-    resolution: {integrity: sha512-C0WQT0gezHuw6AdY1M2jxUO83Rjf0HP7Sk1DtXj6j1EwkQNZrHAg2XPWlq62oqEhYvONq5pkC2Y9oPljWToLmQ==}
+  /typescript/4.9.3:
+    resolution: {integrity: sha512-CIfGzTelbKNEnLpLdGFgdyKhG23CKdKgQPOBc+OUNrkJ2vr+KSzsSV5kq5iWhEQbok+quxgGzrAtGWCyU7tHnA==}
     engines: {node: '>=4.2.0'}
     hasBin: true
 
-  /universalify/0.1.2:
-    resolution: {integrity: sha512-rBJeI5CXAlmy1pV+617WB9J63U6XcazHHF2f2dbJix4XzpUF0RS3Zbj0FGIOCAva5P/d/GBOYaACQ1w+0azUkg==}
+  /universalify/0.2.0:
+    resolution: {integrity: sha512-CJ1QgKmNg3CwvAv/kOFmtnEN05f0D/cn9QntgNOQlQF9dgvVTHj3t+8JPdjqawCHk7V/KA+fbUqzZ9XWhcqPUg==}
     engines: {node: '>= 4.0.0'}
     dev: true
 
@@ -2383,53 +2399,63 @@ packages:
       punycode: 2.1.1
     dev: true
 
-  /util-deprecate/1.0.2:
-    resolution: {integrity: sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==}
+  /url-parse/1.5.10:
+    resolution: {integrity: sha512-WypcfiRhfeUP9vvF0j6rw0J3hrWrw6iZv3+22h6iRMJ/8z1Tj6XfLP4DsUix5MhMPnXpiHDoKyoZ/bdCkwBCiQ==}
+    dependencies:
+      querystringify: 2.2.0
+      requires-port: 1.0.0
     dev: true
 
-  /v8-compile-cache/2.3.0:
-    resolution: {integrity: sha512-l8lCEmLcLYZh4nbunNZvQCJc5pv7+RCwa8q/LdUx8u7lsWvPDKmpodJAJNwkAhJC//dFY48KuIEmjtd4RViDrA==}
+  /util-deprecate/1.0.2:
+    resolution: {integrity: sha512-EPD5q1uXyFxJpCrLnCc1nHnq3gOa6DZBocAIiI2TaSCA7VCJ1UJDMagCzIkXNsUYfD1daK//LTEQ8xiIbrHtcw==}
     dev: true
 
   /v8-to-istanbul/9.0.1:
     resolution: {integrity: sha512-74Y4LqY74kLE6IFyIjPtkSTWzUZmj8tdHT9Ii/26dvQ6K9Dl2NbEfj0XgU2sHCtKgt5VupqhlO/5aWuqS+IY1w==}
     engines: {node: '>=10.12.0'}
     dependencies:
-      '@jridgewell/trace-mapping': 0.3.15
+      '@jridgewell/trace-mapping': 0.3.17
       '@types/istanbul-lib-coverage': 2.0.4
-      convert-source-map: 1.8.0
+      convert-source-map: 1.9.0
     dev: true
 
-  /vite/3.0.8_sass@1.54.4:
-    resolution: {integrity: sha512-AOZ4eN7mrkJiOLuw8IA7piS4IdOQyQCA81GxGsAQvAZzMRi9ZwGB3TOaYsj4uLAWK46T5L4AfQ6InNGlxX30IQ==}
+  /vite/3.2.5_tr7neyh64qpc4cjqb6wcsya4ri:
+    resolution: {integrity: sha512-4mVEpXpSOgrssFZAOmGIr85wPHKvaDAcXqxVxVRZhljkJOMZi1ibLibzjLHzJvcok8BMguLc7g1W6W/GqZbLdQ==}
     engines: {node: ^14.18.0 || >=16.0.0}
     hasBin: true
     peerDependencies:
+      '@types/node': '>= 14'
       less: '*'
       sass: '*'
       stylus: '*'
+      sugarss: '*'
       terser: ^5.4.0
     peerDependenciesMeta:
+      '@types/node':
+        optional: true
       less:
         optional: true
       sass:
         optional: true
       stylus:
         optional: true
+      sugarss:
+        optional: true
       terser:
         optional: true
     dependencies:
-      esbuild: 0.14.54
-      postcss: 8.4.16
+      '@types/node': 18.11.11
+      esbuild: 0.15.18
+      postcss: 8.4.19
       resolve: 1.22.1
-      rollup: 2.77.3
-      sass: 1.54.4
+      rollup: 2.79.1
+      sass: 1.56.1
     optionalDependencies:
       fsevents: 2.3.2
     dev: true
 
-  /vitest/0.22.0_jsdom@20.0.0+sass@1.54.4:
-    resolution: {integrity: sha512-BSIro/QOHLaQY08FHwT6THWhqLQ+VPU+N4Rdo4pcP+16XB6oLmNNAXGcSh/MOLUhfUy+mqCwx7AyKmU7Ms5R+g==}
+  /vitest/0.22.1_jsdom@20.0.3+sass@1.56.1:
+    resolution: {integrity: sha512-+x28YTnSLth4KbXg7MCzoDAzPJlJex7YgiZbUh6YLp0/4PqVZ7q7/zyfdL0OaPtKTpNiQFPpMC8Y2MSzk8F7dw==}
     engines: {node: '>=v14.16.0'}
     hasBin: true
     peerDependencies:
@@ -2450,26 +2476,27 @@ packages:
       jsdom:
         optional: true
     dependencies:
-      '@types/chai': 4.3.3
+      '@types/chai': 4.3.4
       '@types/chai-subset': 1.3.3
-      '@types/node': 18.7.6
-      chai: 4.3.6
+      '@types/node': 18.11.11
+      chai: 4.3.7
       debug: 4.3.4
-      jsdom: 20.0.0
+      jsdom: 20.0.3
       local-pkg: 0.4.2
       tinypool: 0.2.4
-      tinyspy: 1.0.0
-      vite: 3.0.8_sass@1.54.4
+      tinyspy: 1.0.2
+      vite: 3.2.5_tr7neyh64qpc4cjqb6wcsya4ri
     transitivePeerDependencies:
       - less
       - sass
       - stylus
+      - sugarss
       - supports-color
       - terser
     dev: true
 
-  /vue-demi/0.13.8_vue@3.2.37:
-    resolution: {integrity: sha512-Vy1zbZhCOdsmvGR6tJhAvO5vhP7eiS8xkbYQSoVa7o6KlIy3W8Rc53ED4qI4qpeRDjv3mLfXSEpYU6Yq4pgXRg==}
+  /vue-demi/0.13.11_vue@3.2.45:
+    resolution: {integrity: sha512-IR8HoEEGM65YY3ZJYAjMlKygDQn25D5ajNFNoKh9RSDMQtlzCxtfQjdQgv9jjK+m3377SsJXY8ysq8kLCZL25A==}
     engines: {node: '>=12'}
     hasBin: true
     requiresBuild: true
@@ -2480,28 +2507,28 @@ packages:
       '@vue/composition-api':
         optional: true
     dependencies:
-      vue: 3.2.37
+      vue: 3.2.45
     dev: false
 
-  /vue-eslint-parser/9.0.3_eslint@8.22.0:
-    resolution: {integrity: sha512-yL+ZDb+9T0ELG4VIFo/2anAOz8SvBdlqEnQnvJ3M7Scq56DvtjY0VY88bByRZB0D4J0u8olBcfrXTVONXsh4og==}
+  /vue-eslint-parser/9.1.0_eslint@8.29.0:
+    resolution: {integrity: sha512-NGn/iQy8/Wb7RrRa4aRkokyCZfOUWk19OP5HP6JEozQFX5AoS/t+Z0ZN7FY4LlmWc4FNI922V7cvX28zctN8dQ==}
     engines: {node: ^14.17.0 || >=16.0.0}
     peerDependencies:
       eslint: '>=6.0.0'
     dependencies:
       debug: 4.3.4
-      eslint: 8.22.0
+      eslint: 8.29.0
       eslint-scope: 7.1.1
       eslint-visitor-keys: 3.3.0
-      espree: 9.3.3
+      espree: 9.4.1
       esquery: 1.4.0
       lodash: 4.17.21
-      semver: 7.3.7
+      semver: 7.3.8
     transitivePeerDependencies:
       - supports-color
     dev: true
 
-  /vue-i18n/9.2.2_vue@3.2.37:
+  /vue-i18n/9.2.2_vue@3.2.45:
     resolution: {integrity: sha512-yswpwtj89rTBhegUAv9Mu37LNznyu3NpyLQmozF3i1hYOhwpG8RjcjIFIIfnu+2MDZJGSZPXaKWvnQA71Yv9TQ==}
     engines: {node: '>= 14'}
     peerDependencies:
@@ -2510,47 +2537,41 @@ packages:
       '@intlify/core-base': 9.2.2
       '@intlify/shared': 9.2.2
       '@intlify/vue-devtools': 9.2.2
-      '@vue/devtools-api': 6.2.1
-      vue: 3.2.37
+      '@vue/devtools-api': 6.4.5
+      vue: 3.2.45
     dev: false
 
-  /vue-router/4.1.3_vue@3.2.37:
-    resolution: {integrity: sha512-XvK81bcYglKiayT7/vYAg/f36ExPC4t90R/HIpzrZ5x+17BOWptXLCrEPufGgZeuq68ww4ekSIMBZY1qdUdfjA==}
+  /vue-router/4.1.6_vue@3.2.45:
+    resolution: {integrity: sha512-DYWYwsG6xNPmLq/FmZn8Ip+qrhFEzA14EI12MsMgVxvHFDYvlr4NXpVF5hrRH1wVcDP8fGi5F4rxuJSl8/r+EQ==}
     peerDependencies:
       vue: ^3.2.0
     dependencies:
-      '@vue/devtools-api': 6.2.1
-      vue: 3.2.37
+      '@vue/devtools-api': 6.4.5
+      vue: 3.2.45
     dev: false
 
-  /vue-tsc/0.38.9_typescript@4.7.4:
+  /vue-tsc/0.38.9_typescript@4.9.3:
     resolution: {integrity: sha512-Yoy5phgvGqyF98Fb4mYqboR4Q149jrdcGv5kSmufXJUq++RZJ2iMVG0g6zl+v3t4ORVWkQmRpsV4x2szufZ0LQ==}
     hasBin: true
     peerDependencies:
       typescript: '*'
     dependencies:
       '@volar/vue-typescript': 0.38.9
-      typescript: 4.7.4
+      typescript: 4.9.3
     dev: true
 
-  /vue/3.2.37:
-    resolution: {integrity: sha512-bOKEZxrm8Eh+fveCqS1/NkG/n6aMidsI6hahas7pa0w/l7jkbssJVsRhVDs07IdDq7h9KHswZOgItnwJAgtVtQ==}
+  /vue/3.2.45:
+    resolution: {integrity: sha512-9Nx/Mg2b2xWlXykmCwiTUCWHbWIj53bnkizBxKai1g61f2Xit700A1ljowpTIM11e3uipOeiPcSqnmBg6gyiaA==}
     dependencies:
-      '@vue/compiler-dom': 3.2.37
-      '@vue/compiler-sfc': 3.2.37
-      '@vue/runtime-dom': 3.2.37
-      '@vue/server-renderer': 3.2.37_vue@3.2.37
-      '@vue/shared': 3.2.37
+      '@vue/compiler-dom': 3.2.45
+      '@vue/compiler-sfc': 3.2.45
+      '@vue/runtime-dom': 3.2.45
+      '@vue/server-renderer': 3.2.45_vue@3.2.45
+      '@vue/shared': 3.2.45
 
-  /w3c-hr-time/1.0.2:
-    resolution: {integrity: sha512-z8P5DvDNjKDoFIHK7q8r8lackT6l+jo/Ye3HOle7l9nICP9lf1Ci25fy9vHd0JOWewkIFzXIEig3TdKT7JQ5fQ==}
-    dependencies:
-      browser-process-hrtime: 1.0.0
-    dev: true
-
-  /w3c-xmlserializer/3.0.0:
-    resolution: {integrity: sha512-3WFqGEgSXIyGhOmAFtlicJNMjEps8b1MG31NCA0/vOF9+nKMUW1ckhi9cnNHmf88Rzw5V+dwIwsm2C7X8k9aQg==}
-    engines: {node: '>=12'}
+  /w3c-xmlserializer/4.0.0:
+    resolution: {integrity: sha512-d+BFHzbiCx6zGfz0HyQ6Rg69w9k19nviJspaj4yNscGjrHu94sVP+aRm75yEbCh+r2/yR+7q6hux9LVtbuTGBw==}
+    engines: {node: '>=14'}
     dependencies:
       xml-name-validator: 4.0.0
     dev: true
@@ -2606,8 +2627,8 @@ packages:
     resolution: {integrity: sha512-l4Sp/DRseor9wL6EvV2+TuQn63dMkPjZ/sp9XkghTEbV9KlPS1xUsZ3u7/IQO4wxtcFB4bgpQPRcR3QCvezPcQ==}
     dev: true
 
-  /ws/8.8.1:
-    resolution: {integrity: sha512-bGy2JzvzkPowEJV++hF07hAD6niYSr0JzBNo/J29WsB57A2r7Wlc1UFcTR9IzrPvuNVO4B8LGqF8qcpsVOhJCA==}
+  /ws/8.11.0:
+    resolution: {integrity: sha512-HPG3wQd9sNQoT9xHyNCXoDUa+Xw/VevmY9FoHyQ+g+rrMn4j6FB4np7Z0OhdTgjx6MgQLK7jwSy1YecU1+4Asg==}
     engines: {node: '>=10.0.0'}
     peerDependencies:
       bufferutil: ^4.0.1
diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/api/server/index.ts
similarity index 56%
copy from kyuubi-server/web-ui/src/locales/en_US/index.ts
copy to kyuubi-server/web-ui/src/api/server/index.ts
index 92df340d8..431357d14 100644
--- a/kyuubi-server/web-ui/src/locales/en_US/index.ts
+++ b/kyuubi-server/web-ui/src/api/server/index.ts
@@ -15,6 +15,35 @@
  * limitations under the License.
  */
 
-export default {
-  test: 'test'
+import request from '@/utils/request'
+import { IServerSearch, IEngineSearch } from './types'
+
+export function getAllServers(data: IServerSearch) {
+  return request({
+    url: 'api/v1/admin/servers',
+    method: 'get',
+    data
+  })
+}
+
+export function getServerConf() {
+  return request({
+    url: 'api/v1/admin/get/hadoop_conf',
+    method: 'get'
+  })
+}
+
+export function getServerLog() {
+  return request({
+    url: 'api/v1/admin/server/log',
+    method: 'post'
+  })
+}
+
+export function getAllEngines(data: IEngineSearch) {
+  return request({
+    url: 'api/v1/admin/engine',
+    method: 'get',
+    data
+  })
 }
diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/api/server/types.ts
similarity index 80%
copy from kyuubi-server/web-ui/src/locales/en_US/index.ts
copy to kyuubi-server/web-ui/src/api/server/types.ts
index 92df340d8..08d73066c 100644
--- a/kyuubi-server/web-ui/src/locales/en_US/index.ts
+++ b/kyuubi-server/web-ui/src/api/server/types.ts
@@ -15,6 +15,14 @@
  * limitations under the License.
  */
 
-export default {
-  test: 'test'
+interface IServerSearch {
+  host: string | null
 }
+
+interface IEngineSearch {
+  enginetype: null | string
+  sharelevel: null | string
+  subdomain: null | string
+}
+
+export { IServerSearch, IEngineSearch }
diff --git a/kyuubi-server/web-ui/src/api/session/index.ts b/kyuubi-server/web-ui/src/api/session/index.ts
new file mode 100644
index 000000000..9ee2c48d8
--- /dev/null
+++ b/kyuubi-server/web-ui/src/api/session/index.ts
@@ -0,0 +1,76 @@
+/*
+ * 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.
+ */
+
+import request from '@/utils/request'
+import { ISessionSearch, IOperationSearch } from './types'
+
+export function getAllSessions(data: ISessionSearch) {
+  return request({
+    url: 'api/v1/sessions/listSessionInfo',
+    method: 'get',
+    data
+  })
+}
+
+export function getSession(sessionId: string) {
+  return request({
+    url: `api/v1/sessions/${sessionId}`,
+    method: 'get'
+  })
+}
+
+export function deleteSession(sessionId: string) {
+  return request({
+    url: `api/v1/sessions/${sessionId}`,
+    method: 'delete'
+  })
+}
+
+export function getSqlDetails(sessionId: string) {
+  return request({
+    url: `api/v1/sessions/${sessionId}/sqlDetails`,
+    method: 'get'
+  })
+}
+
+export function getAllOperations(data: IOperationSearch) {
+  return request({
+    url: 'api/v1/operations/listOperation',
+    method: 'get',
+    data
+  })
+}
+
+export function cancelOperation(
+  operationId: string,
+  data: {
+    action: 'CANCEL' | 'CLOSE'
+  }
+) {
+  return request({
+    url: `api/v1/operations/${operationId}`,
+    method: 'put',
+    data
+  })
+}
+
+export function getOperationLog(operationId: string) {
+  return request({
+    url: `api/v1/operations/${operationId}/log`,
+    method: 'get'
+  })
+}
diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/api/session/types.ts
similarity index 77%
copy from kyuubi-server/web-ui/src/locales/en_US/index.ts
copy to kyuubi-server/web-ui/src/api/session/types.ts
index 92df340d8..e52afcec7 100644
--- a/kyuubi-server/web-ui/src/locales/en_US/index.ts
+++ b/kyuubi-server/web-ui/src/api/session/types.ts
@@ -15,6 +15,15 @@
  * limitations under the License.
  */
 
-export default {
-  test: 'test'
+interface ISessionSearch {
+  user: string | null
+  serverIP: string | null
 }
+
+interface IOperationSearch {
+  sessionHandle: string | null
+  operationType: string | null
+  state: string | null
+}
+
+export { ISessionSearch, IOperationSearch }
diff --git a/kyuubi-server/web-ui/src/components/menu/index.vue b/kyuubi-server/web-ui/src/components/menu/index.vue
index b563b491e..1a266d74b 100644
--- a/kyuubi-server/web-ui/src/components/menu/index.vue
+++ b/kyuubi-server/web-ui/src/components/menu/index.vue
@@ -56,24 +56,22 @@
   </el-menu>
 </template>
 
-<script lang="ts">
-  export default {
-    name: 'MenuIndex',
-    props: {
-      isCollapse: {
-        type: Boolean,
-        required: true
-      },
-      menus: {
-        type: Array,
-        default: () => []
-      },
-      activePath: {
-        type: String,
-        default: '/overview'
-      }
+<script lang="ts" setup>
+  import { PropType } from 'vue'
+  defineProps({
+    isCollapse: {
+      type: Boolean,
+      required: true
+    },
+    menus: {
+      type: Array as PropType<any[]>,
+      default: () => []
+    },
+    activePath: {
+      type: String,
+      default: '/overview'
     }
-  }
+  })
 </script>
 
 <style lang="scss" scoped>
diff --git a/kyuubi-server/web-ui/src/components/modal/index.vue b/kyuubi-server/web-ui/src/components/modal/index.vue
new file mode 100644
index 000000000..d1b415ca8
--- /dev/null
+++ b/kyuubi-server/web-ui/src/components/modal/index.vue
@@ -0,0 +1,112 @@
+<!--
+* 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.
+-->
+
+<template>
+  <el-dialog
+    v-model="visible"
+    :align-center="true"
+    :title="title"
+    :before-close="handleCancel"
+    class="modal-container"
+  >
+    <slot></slot>
+    <template #footer>
+      <span class="dialog-footer">
+        <el-button
+          v-if="cancelShow"
+          :clsss="confirmClassName"
+          @click="handleCancel"
+          >{{ cancelText || $t('cancel') }}</el-button
+        >
+        <el-button
+          type="primary"
+          :class="cancelClassName"
+          :disabled="confirmDisabled"
+          :loading="confirmLoading"
+          @click="handleConfirm"
+        >
+          {{ confirmText || $t('confirm') }}
+        </el-button>
+      </span>
+    </template>
+  </el-dialog>
+</template>
+
+<script lang="ts" setup>
+  import { PropType, computed } from 'vue'
+
+  const props = defineProps({
+    show: {
+      type: Boolean as PropType<boolean>,
+      default: false
+    },
+    title: {
+      type: String as PropType<string>,
+      required: true
+    },
+    cancelText: {
+      type: String as PropType<string>,
+      default: null
+    },
+    cancelShow: {
+      type: Boolean as PropType<boolean>,
+      default: true
+    },
+    confirmText: {
+      type: String as PropType<string>,
+      default: null
+    },
+    confirmClassName: {
+      type: String as PropType<string>,
+      default: ''
+    },
+    cancelClassName: {
+      type: String as PropType<string>,
+      default: ''
+    },
+    confirmDisabled: {
+      type: Boolean as PropType<boolean>,
+      default: false
+    },
+    confirmLoading: {
+      type: Boolean as PropType<boolean>,
+      default: false
+    }
+  })
+
+  const emit = defineEmits(['cancel', 'confirm'])
+
+  const visible = computed(() => props.show)
+
+  const handleCancel = () => {
+    emit('cancel')
+  }
+
+  const handleConfirm = () => {
+    emit('confirm')
+  }
+</script>
+
+<style lang="scss">
+  .modal-container {
+    .el-dialog__body {
+      max-height: 60vh;
+      overflow: auto;
+    }
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/locales/en_US/index.ts b/kyuubi-server/web-ui/src/locales/en_US/index.ts
index 92df340d8..1134cbe2d 100644
--- a/kyuubi-server/web-ui/src/locales/en_US/index.ts
+++ b/kyuubi-server/web-ui/src/locales/en_US/index.ts
@@ -16,5 +16,44 @@
  */
 
 export default {
-  test: 'test'
+  test: 'test',
+  overview: 'Overview',
+  user: 'User',
+  client_ip: 'Client IP',
+  server_ip: 'Server IP',
+  session_id: 'Session ID',
+  session_name: 'Session Name',
+  running_operations: 'Running Operations',
+  error_operations: 'Error Operations',
+  total_operations: 'Total Operations',
+  start_time: 'Start Time',
+  finish_time: 'Finish Time',
+  duration: 'Duration(hour:min:sec)',
+  operation: 'Operation',
+  statement_id: 'Statement ID',
+  create_time: 'Create Time',
+  statement: 'Statement',
+  engine_id: 'Engine ID',
+  engine_type: 'Engine Type',
+  engine_share_level: 'Engine Share Level',
+  share_level: 'Share Level',
+  failure_reason: 'Failure Reason',
+  operation_id: 'Operation ID',
+  cpu: 'CPU',
+  memory: 'Memory',
+  log: 'Log',
+  view_config: 'View Configuration',
+  engines: 'Engines',
+  status: 'Status',
+  type: 'Type',
+  state: 'State',
+  subdomain: 'Subdomain',
+  confirm: 'Confirm',
+  cancel: 'Cancel',
+  cancel_confirm: 'Cancel?',
+  delete: 'Delete',
+  delete_confirm: 'Delete?',
+  delete_success: 'Delete success',
+  close: 'Close',
+  close_confirm: 'Close?'
 }
diff --git a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts
index 7272109f5..630d17285 100644
--- a/kyuubi-server/web-ui/src/locales/zh_CN/index.ts
+++ b/kyuubi-server/web-ui/src/locales/zh_CN/index.ts
@@ -16,5 +16,46 @@
  */
 
 export default {
-  test: '测试'
+  test: '测试',
+  overview: '概览',
+  user: '用户',
+  client_ip: '客户端IP',
+  server_ip: '服务器IP',
+  session_id: 'Session ID',
+  session_name: 'Session名称',
+  running_operations: '运行操作',
+  error_operations: '错误操作',
+  total_operations: '总共操作',
+  start_time: '开始时间',
+  finish_time: '结束时间',
+  duration: '运行时长(时:分:秒)',
+  operation: '操作',
+  statement_id: 'Statement ID',
+  create_time: '创建时间',
+  statement: 'Statement',
+  engine_id: 'Engine ID',
+  engine_type: 'Engine Type',
+  engine_share_level: 'Engine Share Level',
+  share_level: 'Share Level',
+  failure_reason: '失败原因',
+  operation_id: '操作ID',
+  cpu: 'CPU',
+  memory: '内存',
+  log: '日志',
+  view_config: '查看配置',
+  engines: 'Engines',
+  status: '状态',
+  type: '类型',
+  state: '状态',
+  subdomain: 'Subdomain',
+  confirm: '确定',
+  cancel: '取消',
+  cancel_confirm: '确认取消?',
+  cancel_success: '删除成功',
+  delete: '删除',
+  delete_confirm: '确认删除?',
+  delete_success: '删除成功',
+  close: '关闭',
+  close_confirm: '确认关闭?',
+  close_success: '关闭成功'
 }
diff --git a/kyuubi-server/web-ui/src/router/index.ts b/kyuubi-server/web-ui/src/router/index.ts
index 207a22d56..e95020e95 100644
--- a/kyuubi-server/web-ui/src/router/index.ts
+++ b/kyuubi-server/web-ui/src/router/index.ts
@@ -17,9 +17,9 @@
 
 import { createRouter, createWebHistory } from 'vue-router'
 import overviewRoutes from './overview'
-import workloadRoutes from './workload'
-import operationRoutes from './operation'
-import contactRoutes from './contact'
+import sessionRoutes from './session'
+import serverRoutes from './server'
+import runSqlRoutes from './run-sql'
 
 const routes = [
   {
@@ -36,11 +36,12 @@ const routes = [
     redirect: 'overview',
     children: [
       ...overviewRoutes,
-      ...workloadRoutes,
-      ...operationRoutes,
-      ...contactRoutes
+      ...sessionRoutes,
+      ...serverRoutes,
+      ...runSqlRoutes
     ]
-  }
+  },
+  { path: '/:catchAll(.*)', redirect: '/' }
 ]
 
 const router = createRouter({
diff --git a/kyuubi-server/web-ui/src/router/contact/index.ts b/kyuubi-server/web-ui/src/router/run-sql/index.ts
similarity index 89%
rename from kyuubi-server/web-ui/src/router/contact/index.ts
rename to kyuubi-server/web-ui/src/router/run-sql/index.ts
index a83c653ec..6d7af6e70 100644
--- a/kyuubi-server/web-ui/src/router/contact/index.ts
+++ b/kyuubi-server/web-ui/src/router/run-sql/index.ts
@@ -17,9 +17,9 @@
 
 const routes = [
   {
-    path: '/contact',
-    name: 'contact',
-    component: () => import('@/views/contact/index.vue')
+    path: '/run-sql',
+    name: 'run-sql',
+    component: () => import('@/views/run-sql/index.vue')
   }
 ]
 
diff --git a/kyuubi-server/web-ui/src/router/operation/index.ts b/kyuubi-server/web-ui/src/router/server/index.ts
similarity index 75%
rename from kyuubi-server/web-ui/src/router/operation/index.ts
rename to kyuubi-server/web-ui/src/router/server/index.ts
index 03ba4c285..55984e7df 100644
--- a/kyuubi-server/web-ui/src/router/operation/index.ts
+++ b/kyuubi-server/web-ui/src/router/server/index.ts
@@ -17,14 +17,14 @@
 
 const routes = [
   {
-    path: '/operation/runningJobs',
-    name: 'operation-runningJobs',
-    component: () => import('@/views/operation/runningJobs/index.vue')
+    path: '/server/kyuubi-service',
+    name: 'server-kyuubi-service',
+    component: () => import('@/views/server/kyuubi-service/index.vue')
   },
   {
-    path: '/operation/completedJobs',
-    name: 'operation-completedJobs',
-    component: () => import('@/views/operation/completedJobs/index.vue')
+    path: '/server/engine',
+    name: 'server-engine',
+    component: () => import('@/views/server/engine/index.vue')
   }
 ]
 
diff --git a/kyuubi-server/web-ui/src/router/workload/index.ts b/kyuubi-server/web-ui/src/router/session/index.ts
similarity index 63%
rename from kyuubi-server/web-ui/src/router/workload/index.ts
rename to kyuubi-server/web-ui/src/router/session/index.ts
index 7d7b91a47..7af9b0dbc 100644
--- a/kyuubi-server/web-ui/src/router/workload/index.ts
+++ b/kyuubi-server/web-ui/src/router/session/index.ts
@@ -17,24 +17,19 @@
 
 const routes = [
   {
-    path: '/workload/analysis',
-    name: 'workload-analysis',
-    component: () => import('@/views/workload/analysis/index.vue')
+    path: '/session/session-statistics',
+    name: 'session-statistics',
+    component: () => import('@/views/session/session-statistics/index.vue')
   },
   {
-    path: '/workload/queue',
-    name: 'workload-queue',
-    component: () => import('@/views/workload/queue/index.vue')
+    path: '/session/sql-statistics',
+    name: 'sql-statistics',
+    component: () => import('@/views/session/sql-statistics/index.vue')
   },
   {
-    path: '/workload/session',
-    name: 'workload-session',
-    component: () => import('@/views/workload/session/index.vue')
-  },
-  {
-    path: '/workload/query',
-    name: 'workload-query',
-    component: () => import('@/views/workload/query/index.vue')
+    path: '/session/operation',
+    name: 'operation',
+    component: () => import('@/views/session/operation/index.vue')
   }
 ]
 
diff --git a/kyuubi-server/web-ui/src/styles/element/index.scss b/kyuubi-server/web-ui/src/styles/element/index.scss
index c6aad1578..355c07770 100644
--- a/kyuubi-server/web-ui/src/styles/element/index.scss
+++ b/kyuubi-server/web-ui/src/styles/element/index.scss
@@ -38,7 +38,7 @@ $--colors: (
 
 $--menus: (
   'item-height': 46px,
-  'icon-width': 24px,
+  'icon-width': 12px,
   'text-color': 'rgba(255, 255, 255, .75)',
   'hover-text-color': '#fff',
   'hover-bg-color': '#1890ff',
@@ -46,10 +46,15 @@ $--menus: (
   'active-color': '#1890ff',
 );
 
+$--card: (
+  padding: 14px
+);
+
 @forward 'element-plus/theme-chalk/src/common/var.scss' with (
     // do not use same name, it will override.
     $colors: $--colors,
-    $menu: $--menus
+    $menu: $--menus,
+    $card: $--card
   );
 
 @use "element-plus/theme-chalk/src/index.scss" as *;
diff --git a/kyuubi-server/web-ui/src/styles/index.scss b/kyuubi-server/web-ui/src/styles/index.scss
index 5ae2b5a9e..3cf2c6e5b 100644
--- a/kyuubi-server/web-ui/src/styles/index.scss
+++ b/kyuubi-server/web-ui/src/styles/index.scss
@@ -67,6 +67,16 @@ button:focus-visible {
   outline: 4px auto -webkit-focus-ring-color;
 }
 
+.search-box {
+  > :last-child {
+    margin-right: 0!important;
+  }
+}
+
+.table-container {
+  margin-top: 8px;
+}
+
 #app {
   background: #eee;
   min-height: 100vh;
diff --git a/kyuubi-server/web-ui/src/utils/index.ts b/kyuubi-server/web-ui/src/utils/index.ts
new file mode 100644
index 000000000..98f50b2c9
--- /dev/null
+++ b/kyuubi-server/web-ui/src/utils/index.ts
@@ -0,0 +1,24 @@
+function secondTransfer(val: number) {
+  let theTime = val
+  let middle = 0
+  let hour = 0
+
+  if (theTime >= 60) {
+    middle = Math.floor(theTime / 60)
+    theTime = Math.floor(theTime % 60)
+    if (middle >= 60) {
+      hour = Math.floor(middle / 60)
+      middle = Math.floor(middle % 60)
+    }
+  }
+  let result = String(Math.floor(theTime))
+  if (middle > 0) {
+    result = Math.floor(middle) + ':' + result
+  }
+  if (hour > 0) {
+    result = Math.floor(hour) + ':' + result
+  }
+  return result
+}
+
+export { secondTransfer }
diff --git a/kyuubi-server/web-ui/src/utils/request.ts b/kyuubi-server/web-ui/src/utils/request.ts
index 375e8f9cc..6171ed9cd 100644
--- a/kyuubi-server/web-ui/src/utils/request.ts
+++ b/kyuubi-server/web-ui/src/utils/request.ts
@@ -19,7 +19,7 @@ import axios, { AxiosResponse } from 'axios'
 
 // create an axios instance
 const service = axios.create({
-  baseURL: import.meta.env.VITE_APP_DEV_WEB_URL, // url = base url + request url
+  baseURL: '/', // url = base url + request url
   // withCredentials: true, // send cookies when cross-domain requests
   timeout: 60000 // request timeout
 })
diff --git a/kyuubi-server/web-ui/vite.config.ts b/kyuubi-server/web-ui/src/views/common/use-table.ts
similarity index 61%
copy from kyuubi-server/web-ui/vite.config.ts
copy to kyuubi-server/web-ui/src/views/common/use-table.ts
index 92af99741..71a13a594 100644
--- a/kyuubi-server/web-ui/vite.config.ts
+++ b/kyuubi-server/web-ui/src/views/common/use-table.ts
@@ -15,24 +15,29 @@
  * limitations under the License.
  */
 
-import { defineConfig } from 'vite'
-import Vue from '@vitejs/plugin-vue'
-import path from 'path'
+import { ref, Ref } from 'vue'
 
-export default defineConfig({
-  base: '/ui/',
-  plugins: [Vue()],
-  resolve: {
-    alias: [
-      {
-        find: '@',
-        replacement: path.resolve(__dirname, 'src')
-      },
-      // resolve warning of vue-i18n
-      {
-        find: 'vue-i18n',
-        replacement: 'vue-i18n/dist/vue-i18n.cjs.js'
-      }
-    ]
+export function useTable() {
+  const tableData: Ref<any[]> = ref([])
+  const loading = ref(false)
+  const searchParam = ref()
+
+  const getList = (func: Function, data?: any) => {
+    loading.value = true
+    func(data)
+      .then((res: any[]) => {
+        tableData.value = res || []
+      })
+      .catch(() => (tableData.value = []))
+      .finally(() => {
+        loading.value = false
+      })
+  }
+
+  return {
+    tableData,
+    loading,
+    searchParam,
+    getList
   }
-})
+}
diff --git a/kyuubi-server/web-ui/src/views/layout/components/aside/index.vue b/kyuubi-server/web-ui/src/views/layout/components/aside/index.vue
index 52304abff..64ef90b1e 100644
--- a/kyuubi-server/web-ui/src/views/layout/components/aside/index.vue
+++ b/kyuubi-server/web-ui/src/views/layout/components/aside/index.vue
@@ -25,18 +25,72 @@
 </template>
 
 <script setup lang="ts">
-  import { ref, reactive } from 'vue'
+  import { ref, watch, computed } from 'vue'
   import { useStore } from '@/pinia/layout'
   import { storeToRefs } from 'pinia'
   import { useRoute } from 'vue-router'
-  import { MENUS } from './types'
   import cMenu from '@/components/menu/index.vue'
+  import { useI18n } from 'vue-i18n'
+  const { t, locale } = useI18n()
 
-  const menus = reactive(MENUS)
+  const menus: any = ref([])
   const store = useStore()
   const { isCollapse } = storeToRefs(store)
   const router = useRoute()
-  const activePath = ref(router.path)
+  const activeMenuMap: any = {
+    '/session/sql-statistics': '/session/session-statistics'
+  }
+  const activePath = computed(() => {
+    return activeMenuMap[router.path] || router.path
+  })
+
+  const initMenus = () => {
+    menus.value = [
+      {
+        label: t('overview'),
+        icon: 'Odometer',
+        router: '/overview'
+      },
+      {
+        label: 'Session Management',
+        icon: 'List',
+        children: [
+          {
+            label: 'Session Statistics',
+            router: '/session/session-statistics'
+          },
+          {
+            label: 'Operation',
+            router: '/session/operation'
+          }
+        ]
+      },
+      {
+        label: 'Server Management',
+        icon: 'Coin',
+        children: [
+          {
+            label: 'Kyuubi Server Management',
+            router: '/server/kyuubi-service'
+          },
+          {
+            label: 'Engine Management',
+            router: '/server/engine'
+          }
+        ]
+      },
+      {
+        label: 'Run Sql',
+        icon: 'VideoPlay',
+        router: '/run-sql'
+      }
+    ]
+  }
+
+  watch(locale, () => {
+    initMenus()
+  })
+  initMenus()
 </script>
 
 <style lang="scss" scoped>
diff --git a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts b/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts
deleted file mode 100644
index ddd32bef4..000000000
--- a/kyuubi-server/web-ui/src/views/layout/components/aside/types.ts
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-export const MENUS = [
-  {
-    label: 'Overview',
-    icon: 'Odometer',
-    router: '/overview'
-  },
-  {
-    label: 'Workload',
-    icon: 'List',
-    children: [
-      {
-        label: 'Analysis',
-        icon: 'VideoPlay',
-        router: '/workload/analysis'
-      },
-      {
-        label: 'Queue',
-        icon: 'Select',
-        router: '/workload/queue'
-      },
-      {
-        label: 'Session',
-        icon: 'Select',
-        router: '/workload/session'
-      },
-      {
-        label: 'Query',
-        icon: 'Select',
-        router: '/workload/query'
-      }
-    ]
-  },
-  {
-    label: 'Operation',
-    icon: 'List',
-    children: [
-      {
-        label: 'Running Jobs',
-        icon: 'VideoPlay',
-        router: '/operation/runningJobs'
-      },
-      {
-        label: 'Completed Jobs',
-        icon: 'Select',
-        router: '/operation/completedJobs'
-      }
-    ]
-  },
-  {
-    label: 'Contact Us',
-    icon: 'PhoneFilled',
-    router: '/contact'
-  }
-]
diff --git a/kyuubi-server/web-ui/src/views/layout/index.vue b/kyuubi-server/web-ui/src/views/layout/index.vue
index d5ac5f16e..19241c926 100644
--- a/kyuubi-server/web-ui/src/views/layout/index.vue
+++ b/kyuubi-server/web-ui/src/views/layout/index.vue
@@ -54,7 +54,7 @@
     height: 100%;
 
     .el-container {
-      min-height: 100vh;
+      height: 100vh;
 
       ::v-deep(.el-aside) {
         width: auto;
@@ -71,6 +71,11 @@
         background: #fff;
         box-shadow: 0 1px 4px rgb(0 21 41 / 8%);
       }
+
+      .el-main {
+        height: calc(100% - 64px);
+        overflow: auto;
+      }
     }
   }
 </style>
diff --git a/kyuubi-server/web-ui/src/views/operation/completedJobs/index.vue b/kyuubi-server/web-ui/src/views/operation/completedJobs/index.vue
deleted file mode 100644
index 7b587c4fa..000000000
--- a/kyuubi-server/web-ui/src/views/operation/completedJobs/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Completed jobs page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'OperationCompletedJobsIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/operation/runningJobs/index.vue b/kyuubi-server/web-ui/src/views/operation/runningJobs/index.vue
deleted file mode 100644
index 030b48ae9..000000000
--- a/kyuubi-server/web-ui/src/views/operation/runningJobs/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Running jobs page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'OperationRunningJobsIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/contact/index.vue b/kyuubi-server/web-ui/src/views/run-sql/index.vue
similarity index 88%
rename from kyuubi-server/web-ui/src/views/contact/index.vue
rename to kyuubi-server/web-ui/src/views/run-sql/index.vue
index 86b003f08..d2fcb546f 100644
--- a/kyuubi-server/web-ui/src/views/contact/index.vue
+++ b/kyuubi-server/web-ui/src/views/run-sql/index.vue
@@ -17,13 +17,7 @@
 -->
 
 <template>
-  <main>Contact page</main>
+  <main>Run sql page</main>
 </template>
 
-<script lang="ts">
-  export default {
-    name: 'ContactIndex'
-  }
-</script>
-
 <style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/server/engine/index.vue b/kyuubi-server/web-ui/src/views/server/engine/index.vue
new file mode 100644
index 000000000..1177ad9a4
--- /dev/null
+++ b/kyuubi-server/web-ui/src/views/server/engine/index.vue
@@ -0,0 +1,122 @@
+<!--
+* 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.
+-->
+
+<template>
+  <el-card :body-style="{ padding: '10px 14px' }">
+    <header>
+      <el-space class="search-box">
+        <el-select
+          v-model="searchParam.enginetype"
+          :placeholder="$t('engine_type')"
+          @change="getList"
+        >
+          <el-option
+            v-for="item in ['FLINK_SQL', 'TRINO', 'HIVE_SQL', 'JDBC']"
+            :key="item"
+            :label="item"
+            :value="item"
+          />
+        </el-select>
+        <el-select
+          v-model="searchParam.sharelevel"
+          :placeholder="$t('share_level')"
+          @change="getList"
+        >
+          <el-option
+            v-for="item in ['CONNECTION', 'USER', 'GROUP', 'SERVER']"
+            :key="item"
+            :label="item"
+            :value="item"
+          />
+        </el-select>
+        <el-input
+          v-model="searchParam.subdomain"
+          :placeholder="$t('subdomain')"
+          @keyup.enter="getList"
+        />
+        <el-button type="primary" icon="Search" @click="getList" />
+      </el-space>
+    </header>
+  </el-card>
+  <el-card class="table-container">
+    <el-table v-loading="loading" :data="tableData" style="width: 100%">
+      <el-table-column prop="host" :label="$t('server_ip')" min-width="20%" />
+      <el-table-column
+        prop="engineType"
+        :label="$t('engine_type')"
+        min-width="20%"
+      />
+      <el-table-column
+        prop="sharelevel"
+        :label="$t('share_level')"
+        min-width="20%"
+      />
+      <el-table-column
+        prop="memoryTotal"
+        :label="$t('memory')"
+        min-width="20%"
+      />
+      <el-table-column prop="user" :label="$t('user')" min-width="20%" />
+      <el-table-column :label="$t('start_time')" min-width="20%">
+        <template #default="scope">
+          {{
+            scope.row.createTime != null && Number(scope.row.createTime) > 0
+              ? format(Number(scope.row.createTime), 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column prop="cpuTotal" :label="$t('cpu')" min-width="20%" />
+      <el-table-column
+        prop="memoryTotal"
+        :label="$t('memory')"
+        min-width="20%"
+      />
+      <el-table-column prop="status" :label="$t('status')" min-width="20%" />
+    </el-table>
+  </el-card>
+</template>
+
+<script lang="ts" setup>
+  import { reactive } from 'vue'
+  import { format } from 'date-fns'
+  import { getAllEngines } from '@/api/server'
+  import { IEngineSearch } from '@/api/server/types'
+  import { useTable } from '@/views/common/use-table'
+
+  const searchParam: IEngineSearch = reactive({
+    enginetype: null,
+    sharelevel: null,
+    subdomain: null
+  })
+
+  const { tableData, loading, getList: _getList } = useTable()
+
+  const getList = () => {
+    _getList(getAllEngines, searchParam)
+  }
+
+  getList()
+</script>
+
+<style scoped lang="scss">
+  header {
+    display: flex;
+    justify-content: flex-end;
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/views/server/kyuubi-service/index.vue b/kyuubi-server/web-ui/src/views/server/kyuubi-service/index.vue
new file mode 100644
index 000000000..7e35fec08
--- /dev/null
+++ b/kyuubi-server/web-ui/src/views/server/kyuubi-service/index.vue
@@ -0,0 +1,79 @@
+<!--
+* 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.
+-->
+
+<template>
+  <el-card :body-style="{ padding: '10px 14px' }">
+    <header>
+      <el-space class="search-box">
+        <el-input
+          v-model="searchParam.host"
+          :placeholder="$t('server_ip')"
+          @keyup.enter="getList"
+        />
+        <el-button type="primary" icon="Search" @click="getList" />
+      </el-space>
+    </header>
+  </el-card>
+  <el-card class="table-container">
+    <el-table v-loading="loading" :data="tableData" style="width: 100%">
+      <el-table-column prop="host" :label="$t('server_ip')" min-width="20%" />
+      <el-table-column prop="cpuTotal" :label="$t('cpu')" min-width="20%" />
+      <el-table-column
+        prop="memoryTotal"
+        :label="$t('memory')"
+        min-width="20%"
+      />
+      <el-table-column :label="$t('start_time')" min-width="20%">
+        <template #default="scope">
+          {{
+            scope.row.createTime != null && scope.row.createTime > 0
+              ? format(scope.row.createTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column prop="status" :label="$t('status')" min-width="20%" />
+    </el-table>
+  </el-card>
+</template>
+
+<script lang="ts" setup>
+  import { reactive } from 'vue'
+  import { format } from 'date-fns'
+  import { getAllServers } from '@/api/server'
+  import { useTable } from '@/views/common/use-table'
+
+  const searchParam = reactive({
+    host: null
+  })
+
+  const { tableData, loading, getList: _getList } = useTable()
+
+  const getList = () => {
+    _getList(getAllServers, searchParam)
+  }
+
+  getList()
+</script>
+
+<style scoped lang="scss">
+  header {
+    display: flex;
+    justify-content: flex-end;
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/views/session/operation/index.vue b/kyuubi-server/web-ui/src/views/session/operation/index.vue
new file mode 100644
index 000000000..e8b2a4937
--- /dev/null
+++ b/kyuubi-server/web-ui/src/views/session/operation/index.vue
@@ -0,0 +1,261 @@
+<!--
+* 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.
+-->
+
+<template>
+  <el-card :body-style="{ padding: '10px 14px' }">
+    <header>
+      <el-space class="search-box">
+        <el-input
+          v-model="searchParam.sessionHandle"
+          :placeholder="$t('session_id')"
+          style="width: 300px"
+          @keyup.enter="getList"
+        />
+        <el-select
+          v-model="searchParam.operationType"
+          :placeholder="$t('type')"
+          @change="getList"
+        >
+          <el-option
+            v-for="item in [
+              'EXECUTE_STATEMENT',
+              'GET_TYPE_INFO',
+              'GET_CATALOGS',
+              'GET_SCHEMAS',
+              'GET_TABLES',
+              'GET_TABLE_TYPES',
+              'GET_COLUMNS',
+              'GET_FUNCTIONS',
+              'UNKNOWN'
+            ]"
+            :key="item"
+            :label="item"
+            :value="item"
+          />
+        </el-select>
+        <el-select
+          v-model="searchParam.state"
+          :placeholder="$t('state')"
+          @change="getList"
+        >
+          <el-option
+            v-for="item in [
+              'INITIALIZED_STATE',
+              'PENDING_STATE',
+              'RUNNING_STATE',
+              'TIMEDOUT_STATE',
+              'CANCELED_STATE',
+              'CLOSED_STATE',
+              'ERROR_STATE',
+              'UKNOWN_STATE'
+            ]"
+            :key="item"
+            :label="item"
+            :value="item"
+          />
+        </el-select>
+        <el-button type="primary" icon="Search" @click="getList" />
+      </el-space>
+    </header>
+  </el-card>
+  <el-card class="table-container">
+    <el-table v-loading="loading" :data="tableData" style="width: 100%">
+      <el-table-column prop="sessionId" :label="$t('session_id')" width="160" />
+      <el-table-column
+        prop="statementId"
+        :label="$t('operation_id')"
+        width="160"
+      />
+      <el-table-column prop="operationType" :label="$t('type')" width="160" />
+      <el-table-column prop="state" :label="$t('state')" width="160" />
+      <el-table-column :label="$t('start_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.startTime != null && scope.row.startTime > -1
+              ? format(scope.row.startTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('finish_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.completeTime != null && scope.row.completeTime > -1
+              ? format(scope.row.completeTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('duration')" width="200">
+        <template #default="scope">{{
+          scope.row.startTime != null &&
+          scope.row.completeTime != null &&
+          scope.row.startTime > -1 &&
+          scope.row.completeTime > -1
+            ? secondTransfer(
+                (scope.row.completeTime - scope.row.startTime) / 1000
+              )
+            : '-'
+        }}</template>
+      </el-table-column>
+      <el-table-column fixed="right" :label="$t('operation')" width="144">
+        <template #default="scope">
+          <el-space wrap>
+            <el-popconfirm
+              :title="$t('cancel_confirm')"
+              @confirm="handleOperate(scope.row.statementId, 'CANCEL')"
+            >
+              <template #reference>
+                <span>
+                  <el-tooltip
+                    effect="dark"
+                    :content="$t('cancel')"
+                    placement="top"
+                  >
+                    <template #default>
+                      <el-button type="danger" icon="Remove" circle />
+                    </template>
+                  </el-tooltip>
+                </span>
+              </template>
+            </el-popconfirm>
+            <el-popconfirm
+              :title="$t('close_confirm')"
+              @confirm="handleOperate(scope.row.statementId, 'CLOSE')"
+            >
+              <template #reference>
+                <span>
+                  <el-tooltip
+                    effect="dark"
+                    :content="$t('close')"
+                    placement="top"
+                  >
+                    <template #default>
+                      <el-button type="danger" icon="CircleClose" circle />
+                    </template>
+                  </el-tooltip>
+                </span>
+              </template>
+            </el-popconfirm>
+            <el-tooltip effect="dark" :content="$t('log')" placement="top">
+              <el-button
+                type="primary"
+                icon="Tickets"
+                circle
+                @click="openLogModal(scope.row.statementId)"
+              />
+            </el-tooltip>
+          </el-space>
+        </template>
+      </el-table-column>
+    </el-table>
+  </el-card>
+  <Modal
+    v-if="logModalVisible"
+    :show="logModalVisible"
+    :title="$t('log')"
+    :cancel-show="false"
+    @cancel="cancelLogModal"
+    @confirm="cancelLogModal"
+  >
+    <div v-loading="logLoading" class="log">{{ log }}</div>
+  </Modal>
+</template>
+
+<script lang="ts" setup>
+  import { ref, reactive } from 'vue'
+  import { useRoute } from 'vue-router'
+  import {
+    getAllOperations,
+    getOperationLog,
+    cancelOperation
+  } from '@/api/session'
+  import { IOperationSearch } from '@/api/session/types'
+  import { format } from 'date-fns'
+  import { useI18n } from 'vue-i18n'
+  import { ElMessage } from 'element-plus'
+  import { secondTransfer } from '@/utils'
+  import { useTable } from '@/views/common/use-table'
+  import Modal from '@/components/modal/index.vue'
+
+  const logModalVisible = ref(false)
+  const log = ref()
+  const logLoading = ref(false)
+  const searchParam: IOperationSearch = reactive({
+    sessionHandle: null,
+    operationType: null,
+    state: null
+  })
+  const route = useRoute()
+  const { t } = useI18n()
+  searchParam.sessionHandle =
+    route.query.sessionId === '' || route.query.sessionId == null
+      ? null
+      : (route.query.sessionId as string)
+  const { tableData, loading, getList: _getList } = useTable()
+
+  const openLogModal = (id: string) => {
+    logModalVisible.value = true
+    logLoading.value = true
+    getOperationLog(id)
+      .then((res: any) => {
+        log.value = Array.isArray(res.logRowSet) ? res.logRowSet.join('\n') : ''
+      })
+      .catch(() => {
+        log.value = ''
+      })
+      .finally(() => {
+        logLoading.value = false
+      })
+  }
+
+  const cancelLogModal = () => {
+    logModalVisible.value = false
+  }
+
+  const handleOperate = (
+    operationId: string,
+    operation: 'CANCEL' | 'CLOSE'
+  ) => {
+    cancelOperation(operationId, { action: operation }).then(() => {
+      // need add delete success or failed logic after api support
+      ElMessage({
+        message: t(`${operation.toLowerCase()}_success`),
+        type: 'success'
+      })
+      getList()
+    })
+  }
+
+  const getList = () => {
+    _getList(getAllOperations, searchParam)
+  }
+
+  getList()
+</script>
+<style lang="scss" scoped>
+  header {
+    display: flex;
+    justify-content: flex-end;
+  }
+
+  .log {
+    font-family: monospace;
+    white-space: pre-line;
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/views/session/session-statistics/index.vue b/kyuubi-server/web-ui/src/views/session/session-statistics/index.vue
new file mode 100644
index 000000000..8f65ecdd5
--- /dev/null
+++ b/kyuubi-server/web-ui/src/views/session/session-statistics/index.vue
@@ -0,0 +1,172 @@
+<!--
+* 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.
+-->
+
+<template>
+  <el-card :body-style="{ padding: '10px 14px' }">
+    <header>
+      <el-space class="search-box">
+        <el-input
+          v-model="searchParam.user"
+          :placeholder="$t('user')"
+          @keyup.enter="getList"
+        />
+        <el-input
+          v-model="searchParam.serverIP"
+          :placeholder="$t('server_ip')"
+          @keyup.enter="getList"
+        />
+        <el-button type="primary" icon="Search" @click="getList" />
+      </el-space>
+    </header>
+  </el-card>
+  <el-card class="table-container">
+    <el-table v-loading="loading" :data="tableData" style="width: 100%">
+      <el-table-column prop="user" :label="$t('user')" width="160" />
+      <el-table-column prop="clientIP" :label="$t('client_ip')" width="160" />
+      <el-table-column prop="serverIP" :label="$t('server_ip')" width="180" />
+      <el-table-column :label="$t('session_id')" width="160">
+        <template #default="scope">
+          <el-link @click="handleSessionJump(scope.row.sessionId)">{{
+            scope.row.sessionId
+          }}</el-link>
+        </template>
+      </el-table-column>
+      <el-table-column
+        prop="sessionName"
+        :label="$t('session_name')"
+        width="160"
+      />
+      <el-table-column
+        prop="runningOperations"
+        :label="$t('running_operations')"
+        width="120"
+      />
+      <el-table-column
+        prop="errorOperations"
+        :label="$t('error_operations')"
+        width="120"
+      />
+      <el-table-column
+        prop="totalOperations"
+        :label="$t('total_operations')"
+        width="120"
+      />
+      <el-table-column :label="$t('start_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.startTime != null && scope.row.startTime > -1
+              ? format(scope.row.startTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('finish_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.endTime != null && scope.row.endTime > -1
+              ? format(scope.row.endTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('duration')" width="200">
+        <template #default="scope">{{
+          scope.row.startTime != null &&
+          scope.row.endTime != null &&
+          scope.row.startTime > -1 &&
+          scope.row.endTime > -1
+            ? secondTransfer((scope.row.endTime - scope.row.startTime) / 1000)
+            : '-'
+        }}</template>
+      </el-table-column>
+      <el-table-column fixed="right" :label="$t('operation')" width="120">
+        <template #default="scope">
+          <el-popconfirm
+            :title="$t('delete_confirm')"
+            @confirm="handleDeleteSession(scope.row.sessionId)"
+          >
+            <template #reference>
+              <span>
+                <el-tooltip
+                  effect="dark"
+                  :content="$t('delete')"
+                  placement="top"
+                >
+                  <template #default>
+                    <el-button type="danger" icon="Delete" circle />
+                  </template> </el-tooltip
+              ></span>
+            </template>
+          </el-popconfirm>
+        </template>
+      </el-table-column>
+    </el-table>
+  </el-card>
+</template>
+
+<script lang="ts" setup>
+  import { reactive } from 'vue'
+  import { format } from 'date-fns'
+  import { secondTransfer } from '@/utils'
+  import { getAllSessions, deleteSession } from '@/api/session'
+  import { Router, useRouter } from 'vue-router'
+  import { ElMessage } from 'element-plus'
+  import { useI18n } from 'vue-i18n'
+  import { useTable } from '@/views/common/use-table'
+
+  const router: Router = useRouter()
+  const { t } = useI18n()
+  const searchParam = reactive({
+    user: null,
+    serverIP: null
+  })
+  const { tableData, loading, getList: _getList } = useTable()
+
+  const handleSessionJump = (sessionId: string) => {
+    router.push({
+      path: '/session/sql-statistics',
+      query: {
+        sessionId
+      }
+    })
+  }
+
+  const handleDeleteSession = (sessionId: string) => {
+    deleteSession(sessionId).then(() => {
+      // need add delete success or failed logic after api support
+      getList()
+      ElMessage({
+        message: t('delete_success'),
+        type: 'success'
+      })
+    })
+  }
+
+  const getList = () => {
+    _getList(getAllSessions, searchParam)
+  }
+
+  getList()
+</script>
+
+<style scoped lang="scss">
+  header {
+    display: flex;
+    justify-content: flex-end;
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/views/session/sql-statistics/index.vue b/kyuubi-server/web-ui/src/views/session/sql-statistics/index.vue
new file mode 100644
index 000000000..bd88ba450
--- /dev/null
+++ b/kyuubi-server/web-ui/src/views/session/sql-statistics/index.vue
@@ -0,0 +1,203 @@
+<!--
+* 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.
+-->
+<template>
+  <el-card :body-style="{ padding: '10px 14px' }">
+    <header>
+      <el-breadcrumb separator="/">
+        <el-breadcrumb-item :to="{ path: '/session/session-statistics' }"
+          >Session Statistics</el-breadcrumb-item
+        >
+        <el-breadcrumb-item>Sql Statistics</el-breadcrumb-item>
+      </el-breadcrumb>
+    </header>
+  </el-card>
+  <el-card
+    v-loading="sessionPropertiesLoading"
+    class="table-container session-properties-container"
+  >
+    <template #header>
+      <div class="card-header">
+        <span>Session Properties</span>
+      </div>
+    </template>
+    <div class="main">
+      <div
+        v-for="(p, key) in sessionProperties"
+        :key="key"
+        class="session-property"
+        ><el-tag>{{ `${key} : ${p}` }}</el-tag></div
+      >
+    </div>
+  </el-card>
+  <el-card class="table-container">
+    <template #header>
+      <div class="card-header">
+        <span>Sql Details</span>
+      </div>
+    </template>
+    <el-table
+      v-loading="sqlDetailsLoading"
+      :data="sqlDetails"
+      style="width: 100%"
+    >
+      <el-table-column prop="sessionUser" :label="$t('user')" width="160" />
+      <el-table-column
+        prop="statementId"
+        :label="$t('statement_id')"
+        width="160"
+      />
+      <el-table-column :label="$t('create_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.createTime != null && scope.row.createTime > -1
+              ? format(scope.row.createTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('finish_time')" width="200">
+        <template #default="scope">
+          {{
+            scope.row.completeTime != null && scope.row.completeTime > -1
+              ? format(scope.row.completeTime, 'yyyy-MM-dd HH:mm:ss')
+              : '-'
+          }}
+        </template>
+      </el-table-column>
+      <el-table-column :label="$t('duration')" width="200">
+        <template #default="scope">{{
+          scope.row.createTime != null &&
+          scope.row.completeTime != null &&
+          scope.row.createTime > -1 &&
+          scope.row.completeTime > -1
+            ? secondTransfer(
+                (scope.row.completeTime - scope.row.createTime) / 1000
+              )
+            : '-'
+        }}</template>
+      </el-table-column>
+      <el-table-column prop="statement" :label="$t('statement')" width="160" />
+      <el-table-column prop="engineId" :label="$t('engine_id')" width="160" />
+      <el-table-column
+        prop="engineType"
+        :label="$t('engine_type')"
+        width="160"
+      />
+      <el-table-column
+        prop="engineShareLevel"
+        :label="$t('engine_share_level')"
+        width="160"
+      />
+      <el-table-column
+        prop="exception"
+        :label="$t('failure_reason')"
+        width="160"
+      />
+      <el-table-column fixed="right" :label="$t('operation')" width="120">
+        <template #default="scope">
+          <el-tooltip effect="dark" :content="$t('operation')" placement="top">
+            <el-button
+              type="primary"
+              icon="Operation"
+              circle
+              @click="openOperationPage(scope.row.sessionId)"
+            />
+          </el-tooltip>
+        </template>
+      </el-table-column>
+    </el-table>
+  </el-card>
+</template>
+
+<script lang="ts" setup>
+  import { Ref, ref } from 'vue'
+  import { getSession, getSqlDetails } from '@/api/session'
+  import { useRoute, useRouter, Router } from 'vue-router'
+  import { format } from 'date-fns'
+  import { secondTransfer } from '@/utils'
+
+  const route = useRoute()
+  const router: Router = useRouter()
+  const sessionProperties: Ref<any> = ref({})
+  const sessionPropertiesLoading = ref(false)
+  const sqlDetails = ref([])
+  const sqlDetailsLoading = ref(false)
+
+  const openOperationPage = (sessionId: string) => {
+    router.push({
+      path: '/session/operation',
+      query: {
+        sessionId
+      }
+    })
+  }
+
+  const getSessionById = () => {
+    const sessionId = route.query.sessionId
+    if (sessionId) {
+      sessionPropertiesLoading.value = true
+      getSession(sessionId as string)
+        .then((res: any) => {
+          sessionProperties.value = res?.conf || {}
+        })
+        .finally(() => {
+          sessionPropertiesLoading.value = false
+        })
+    }
+  }
+
+  const getSqlDetailsById = () => {
+    const sessionId = route.query.sessionId
+    if (sessionId) {
+      sqlDetailsLoading.value = true
+      getSqlDetails(sessionId as string)
+        .then((res: any) => {
+          sqlDetails.value = res || []
+        })
+        .finally(() => {
+          sqlDetailsLoading.value = false
+        })
+    }
+  }
+
+  getSessionById()
+  getSqlDetailsById()
+</script>
+<style lang="scss" scoped>
+  header {
+    display: flex;
+    justify-content: space-between;
+    .el-breadcrumb {
+      line-height: 32px;
+    }
+  }
+
+  .session-properties-container {
+    max-height: 400px;
+    overflow: auto;
+
+    .main {
+      display: flex;
+      flex-wrap: wrap;
+      .session-property {
+        margin-bottom: 6px;
+        margin-right: 6px;
+      }
+    }
+  }
+</style>
diff --git a/kyuubi-server/web-ui/src/views/workload/analysis/index.vue b/kyuubi-server/web-ui/src/views/workload/analysis/index.vue
deleted file mode 100644
index 31b42d46e..000000000
--- a/kyuubi-server/web-ui/src/views/workload/analysis/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Analysis page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'WorkloadAnalysisIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/workload/query/index.vue b/kyuubi-server/web-ui/src/views/workload/query/index.vue
deleted file mode 100644
index 45d0cd91b..000000000
--- a/kyuubi-server/web-ui/src/views/workload/query/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Query page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'WorkloadQueryIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/workload/queue/index.vue b/kyuubi-server/web-ui/src/views/workload/queue/index.vue
deleted file mode 100644
index bbeb8e985..000000000
--- a/kyuubi-server/web-ui/src/views/workload/queue/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Queue page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'WorkloadQueueIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/src/views/workload/session/index.vue b/kyuubi-server/web-ui/src/views/workload/session/index.vue
deleted file mode 100644
index bd4ec51d5..000000000
--- a/kyuubi-server/web-ui/src/views/workload/session/index.vue
+++ /dev/null
@@ -1,29 +0,0 @@
-<!--
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
--->
-
-<template>
-  <main>Session page</main>
-</template>
-
-<script lang="ts">
-  export default {
-    name: 'WorkloadSessionIndex'
-  }
-</script>
-
-<style scoped></style>
diff --git a/kyuubi-server/web-ui/vite.config.ts b/kyuubi-server/web-ui/vite.config.ts
index 92af99741..6d284b59d 100644
--- a/kyuubi-server/web-ui/vite.config.ts
+++ b/kyuubi-server/web-ui/vite.config.ts
@@ -34,5 +34,14 @@ export default defineConfig({
         replacement: 'vue-i18n/dist/vue-i18n.cjs.js'
       }
     ]
+  },
+  server: {
+    // 设置代理
+    proxy: {
+      '/api': {
+        target: 'http://localhost:10099',
+        changeOrigin: true,
+      }
+    }
   }
 })
diff --git a/pom.xml b/pom.xml
index 2ebed48b3..151b2986c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1078,6 +1078,12 @@
                 <version>${jetty.version}</version>
             </dependency>
 
+            <dependency>
+                <groupId>org.eclipse.jetty</groupId>
+                <artifactId>jetty-proxy</artifactId>
+                <version>${jetty.version}</version>
+            </dependency>
+
             <dependency>
                 <groupId>org.scalatest</groupId>
                 <artifactId>scalatest_${scala.binary.version}</artifactId>