You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ul...@apache.org on 2022/09/09 01:59:03 UTC

[incubator-kyuubi] branch master updated: [KYUUBI #3309] Add configs of engine event logger for each engine

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 9440d1b70 [KYUUBI #3309] Add configs of engine event logger for each engine
9440d1b70 is described below

commit 9440d1b7088eef4268d0f3dc843e8c8ed6566e77
Author: Min <zh...@163.com>
AuthorDate: Fri Sep 9 09:58:50 2022 +0800

    [KYUUBI #3309] Add configs of engine event logger for each engine
    
    ### _Why are the changes needed?_
    
    The default value of kyuubi.engine.event.loggers is SPARK, but it can't be used outside the spark engine, so we add the config for each engines.
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #3309 from zhaomin1423/add_engine_logger_conf.
    
    Closes #3309
    
    37b3c05f [Xiao Zhao] fix
    b22356f2 [Min] fix typo
    431bb02b [Min] update settings.md
    44475f46 [Min] Add configs of engine event logger for each engine
    
    Lead-authored-by: Min <zh...@163.com>
    Co-authored-by: Xiao Zhao <zh...@163.com>
    Signed-off-by: ulysses-you <ul...@apache.org>
---
 docs/deployment/settings.md                        |  3 ++
 .../apache/kyuubi/engine/hive/HiveSQLEngine.scala  |  4 +-
 .../hive/events/HiveEventHandlerRegister.scala     |  6 ++-
 .../kyuubi/engine/spark/SparkSQLEngine.scala       |  2 +-
 .../spark/events/SparkEventHandlerRegister.scala   |  5 ++-
 .../SparkJsonLoggingEventHandlerSuite.scala        |  2 +-
 .../kyuubi/engine/trino/TrinoSqlEngine.scala       |  4 +-
 .../trino/event/TrinoEventHandlerRegister.scala    |  6 ++-
 .../org/apache/kyuubi/config/KyuubiConf.scala      | 47 ++++++++++++++++++++++
 .../kyuubi/events/EventHandlerRegister.scala       | 10 ++---
 .../kyuubi/events/ServerEventHandlerRegister.scala |  6 ++-
 .../org/apache/kyuubi/server/KyuubiServer.scala    |  2 +-
 .../ServerJsonLoggingEventHandlerSuite.scala       |  2 +-
 13 files changed, 78 insertions(+), 21 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index 2020eddcf..ed3f134f0 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -226,6 +226,7 @@ kyuubi.engine.event.loggers|SPARK|A comma separated list of engine history logge
 kyuubi.engine.flink.extra.classpath|&lt;undefined&gt;|The extra classpath for the flink sql engine, for configuring location of hadoop client jars, etc|string|1.6.0
 kyuubi.engine.flink.java.options|&lt;undefined&gt;|The extra java options for the flink sql engine|string|1.6.0
 kyuubi.engine.flink.memory|1g|The heap memory for the flink sql engine|string|1.6.0
+kyuubi.engine.hive.event.loggers|JSON|A comma separated list of engine history loggers, where engine/session/operation etc events go. We use spark logger by default.<ul> <li>JSON: the events will be written to the location of kyuubi.engine.event.json.log.path</li> <li>JDBC: to be done</li> <li>CUSTOM: to be done.</li></ul>|seq|1.7.0
 kyuubi.engine.hive.extra.classpath|&lt;undefined&gt;|The extra classpath for the hive query engine, for configuring location of hadoop client jars, etc|string|1.6.0
 kyuubi.engine.hive.java.options|&lt;undefined&gt;|The extra java options for the hive query engine|string|1.6.0
 kyuubi.engine.hive.memory|1g|The heap memory for the hive query engine|string|1.6.0
@@ -250,6 +251,8 @@ kyuubi.engine.share.level|USER|Engines will be shared in different levels, avail
 kyuubi.engine.share.level.sub.domain|&lt;undefined&gt;|(deprecated) - Using kyuubi.engine.share.level.subdomain instead|string|1.2.0
 kyuubi.engine.share.level.subdomain|&lt;undefined&gt;|Allow end-users to create a subdomain for the share level of an engine. A subdomain is a case-insensitive string values that must be a valid zookeeper sub path. For example, for `USER` share level, an end-user can share a certain engine within a subdomain, not for all of its clients. End-users are free to create multiple engines in the `USER` share level. When disable engine pool, use 'default' if absent.|string|1.4.0
 kyuubi.engine.single.spark.session|false|When set to true, this engine is running in a single session mode. All the JDBC/ODBC connections share the temporary views, function registries, SQL configuration and the current database.|boolean|1.3.0
+kyuubi.engine.spark.event.loggers|SPARK|A comma separated list of engine loggers, where engine/session/operation etc events go. We use spark logger by default.<ul> <li>SPARK: the events will be written to the spark listener bus.</li> <li>JSON: the events will be written to the location of kyuubi.engine.event.json.log.path</li> <li>JDBC: to be done</li> <li>CUSTOM: to be done.</li></ul>|seq|1.7.0
+kyuubi.engine.trino.event.loggers|JSON|A comma separated list of engine history loggers, where engine/session/operation etc events go. We use spark logger by default.<ul> <li>JSON: the events will be written to the location of kyuubi.engine.event.json.log.path</li> <li>JDBC: to be done</li> <li>CUSTOM: to be done.</li></ul>|seq|1.7.0
 kyuubi.engine.trino.extra.classpath|&lt;undefined&gt;|The extra classpath for the trino query engine, for configuring other libs which may need by the trino engine |string|1.6.0
 kyuubi.engine.trino.java.options|&lt;undefined&gt;|The extra java options for the trino query engine|string|1.6.0
 kyuubi.engine.trino.memory|1g|The heap memory for the trino query engine|string|1.6.0
diff --git a/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/HiveSQLEngine.scala b/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/HiveSQLEngine.scala
index 3b99c2f22..73132d5fc 100644
--- a/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/HiveSQLEngine.scala
+++ b/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/HiveSQLEngine.scala
@@ -26,7 +26,6 @@ import org.apache.hadoop.security.UserGroupInformation
 
 import org.apache.kyuubi.{Logging, Utils}
 import org.apache.kyuubi.config.{KyuubiConf, KyuubiReservedKeys}
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_LOGGERS
 import org.apache.kyuubi.engine.hive.HiveSQLEngine.currentEngine
 import org.apache.kyuubi.engine.hive.events.{HiveEngineEvent, HiveEventHandlerRegister}
 import org.apache.kyuubi.events.EventBus
@@ -66,7 +65,6 @@ object HiveSQLEngine extends Logging {
   var currentEngine: Option[HiveSQLEngine] = None
   val hiveConf = new HiveConf()
   val kyuubiConf = new KyuubiConf()
-  kyuubiConf.set(ENGINE_EVENT_LOGGERS.key, "JSON")
 
   def startEngine(): Unit = {
     try {
@@ -113,7 +111,7 @@ object HiveSQLEngine extends Logging {
   }
 
   private def initLoggerEventHandler(conf: KyuubiConf): Unit = {
-    HiveEventHandlerRegister.registerEngineEventLoggers(conf)
+    HiveEventHandlerRegister.registerEventLoggers(conf)
   }
 
   def main(args: Array[String]): Unit = {
diff --git a/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/events/HiveEventHandlerRegister.scala b/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/events/HiveEventHandlerRegister.scala
index f34a74fa0..fcd2ec6ee 100644
--- a/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/events/HiveEventHandlerRegister.scala
+++ b/externals/kyuubi-hive-sql-engine/src/main/scala/org/apache/kyuubi/engine/hive/events/HiveEventHandlerRegister.scala
@@ -19,7 +19,7 @@ package org.apache.kyuubi.engine.hive.events
 import java.net.InetAddress
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, ENGINE_HIVE_EVENT_LOGGERS}
 import org.apache.kyuubi.engine.hive.events.handler.HiveJsonLoggingEventHandler
 import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
 import org.apache.kyuubi.events.handler.EventHandler
@@ -37,4 +37,8 @@ object HiveEventHandlerRegister extends EventHandlerRegister {
       hadoopConf,
       kyuubiConf)
   }
+
+  override protected def getLoggers(conf: KyuubiConf): Seq[String] = {
+    conf.get(ENGINE_HIVE_EVENT_LOGGERS)
+  }
 }
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
index 0244ef269..bd8486689 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
@@ -238,7 +238,7 @@ object SparkSQLEngine extends Logging {
 
     def initLoggerEventHandler(conf: KyuubiConf): Unit = {
       val sparkEventRegister = new SparkEventHandlerRegister(spark)
-      sparkEventRegister.registerEngineEventLoggers(conf)
+      sparkEventRegister.registerEventLoggers(conf)
     }
   }
 
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventHandlerRegister.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventHandlerRegister.scala
index 0eeb3b57b..07bd56280 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventHandlerRegister.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventHandlerRegister.scala
@@ -19,7 +19,7 @@ package org.apache.kyuubi.engine.spark.events
 import org.apache.spark.sql.SparkSession
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, ENGINE_SPARK_EVENT_LOGGERS}
 import org.apache.kyuubi.engine.spark.events.handler.{SparkHistoryLoggingEventHandler, SparkJsonLoggingEventHandler}
 import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
 import org.apache.kyuubi.events.handler.EventHandler
@@ -42,4 +42,7 @@ class SparkEventHandlerRegister(spark: SparkSession) extends EventHandlerRegiste
       kyuubiConf)
   }
 
+  override protected def getLoggers(conf: KyuubiConf): Seq[String] = {
+    conf.get(ENGINE_SPARK_EVENT_LOGGERS)
+  }
 }
diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/handler/SparkJsonLoggingEventHandlerSuite.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/handler/SparkJsonLoggingEventHandlerSuite.scala
index 2f24daff0..ddaa96219 100644
--- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/handler/SparkJsonLoggingEventHandlerSuite.scala
+++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/handler/SparkJsonLoggingEventHandlerSuite.scala
@@ -39,7 +39,7 @@ class SparkJsonLoggingEventHandlerSuite extends WithSparkSQLEngine with HiveJDBC
   private val currentDate = Utils.getDateFromTimestamp(System.currentTimeMillis())
 
   override def withKyuubiConf: Map[String, String] = Map(
-    KyuubiConf.ENGINE_EVENT_LOGGERS.key -> s"$JSON",
+    KyuubiConf.ENGINE_SPARK_EVENT_LOGGERS.key -> s"$JSON",
     KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH.key -> logRoot,
     "spark.eventLog.enabled" -> "true",
     "spark.eventLog.dir" -> logRoot)
diff --git a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoSqlEngine.scala b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoSqlEngine.scala
index ca92da04a..5037a882a 100644
--- a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoSqlEngine.scala
+++ b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/TrinoSqlEngine.scala
@@ -24,7 +24,6 @@ import scala.util.control.NonFatal
 import org.apache.kyuubi.{Logging, Utils}
 import org.apache.kyuubi.Utils.{addShutdownHook, TRINO_ENGINE_SHUTDOWN_PRIORITY}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_LOGGERS
 import org.apache.kyuubi.engine.trino.TrinoSqlEngine.{countDownLatch, currentEngine}
 import org.apache.kyuubi.engine.trino.event.{TrinoEngineEvent, TrinoEventHandlerRegister}
 import org.apache.kyuubi.events.EventBus
@@ -59,7 +58,6 @@ object TrinoSqlEngine extends Logging {
   private val countDownLatch = new CountDownLatch(1)
 
   val kyuubiConf: KyuubiConf = KyuubiConf()
-    .set(ENGINE_EVENT_LOGGERS.key, "JSON")
 
   var currentEngine: Option[TrinoSqlEngine] = None
 
@@ -88,7 +86,7 @@ object TrinoSqlEngine extends Logging {
   }
 
   private def initLoggerEventHandler(conf: KyuubiConf): Unit = {
-    TrinoEventHandlerRegister.registerEngineEventLoggers(conf)
+    TrinoEventHandlerRegister.registerEventLoggers(conf)
   }
 
   def main(args: Array[String]): Unit = {
diff --git a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/event/TrinoEventHandlerRegister.scala b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/event/TrinoEventHandlerRegister.scala
index 98b42d9bf..9407af58e 100644
--- a/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/event/TrinoEventHandlerRegister.scala
+++ b/externals/kyuubi-trino-engine/src/main/scala/org/apache/kyuubi/engine/trino/event/TrinoEventHandlerRegister.scala
@@ -19,7 +19,7 @@ package org.apache.kyuubi.engine.trino.event
 import java.net.InetAddress
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
+import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, ENGINE_TRINO_EVENT_LOGGERS}
 import org.apache.kyuubi.engine.trino.event.handler.TrinoJsonLoggingEventHandler
 import org.apache.kyuubi.events.{EventHandlerRegister, KyuubiEvent}
 import org.apache.kyuubi.events.handler.EventHandler
@@ -37,4 +37,8 @@ object TrinoEventHandlerRegister extends EventHandlerRegister {
       hadoopConf,
       kyuubiConf)
   }
+
+  override protected def getLoggers(conf: KyuubiConf): Seq[String] = {
+    conf.get(ENGINE_TRINO_EVENT_LOGGERS)
+  }
 }
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 4aa6c0912..f30b1260b 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
@@ -1492,6 +1492,7 @@ object KyuubiConf {
       .checkValue(_.toSet.subsetOf(Set("JSON", "JDBC", "CUSTOM")), "Unsupported event loggers")
       .createWithDefault(Nil)
 
+  @deprecated("using kyuubi.engine.spark.event.loggers instead", "1.6.0")
   val ENGINE_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
     buildConf("kyuubi.engine.event.loggers")
       .doc("A comma separated list of engine history loggers, where engine/session/operation etc" +
@@ -2040,4 +2041,50 @@ object KyuubiConf {
       .version("1.6.0")
       .stringConf
       .createOptional
+
+  val ENGINE_SPARK_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.engine.spark.event.loggers")
+      .doc("A comma separated list of engine loggers, where engine/session/operation etc" +
+        " events go. We use spark logger by default.<ul>" +
+        " <li>SPARK: the events will be written to the spark listener bus.</li>" +
+        " <li>JSON: the events will be written to the location of" +
+        s" ${ENGINE_EVENT_JSON_LOG_PATH.key}</li>" +
+        " <li>JDBC: to be done</li>" +
+        " <li>CUSTOM: to be done.</li></ul>")
+      .version("1.7.0")
+      .fallbackConf(ENGINE_EVENT_LOGGERS)
+
+  val ENGINE_HIVE_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.engine.hive.event.loggers")
+      .doc("A comma separated list of engine history loggers, where engine/session/operation etc" +
+        " events go. We use spark logger by default.<ul>" +
+        " <li>JSON: the events will be written to the location of" +
+        s" ${ENGINE_EVENT_JSON_LOG_PATH.key}</li>" +
+        " <li>JDBC: to be done</li>" +
+        " <li>CUSTOM: to be done.</li></ul>")
+      .version("1.7.0")
+      .stringConf
+      .transform(_.toUpperCase(Locale.ROOT))
+      .toSequence()
+      .checkValue(
+        _.toSet.subsetOf(Set("JSON", "JDBC", "CUSTOM")),
+        "Unsupported event loggers")
+      .createWithDefault(Seq("JSON"))
+
+  val ENGINE_TRINO_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
+    buildConf("kyuubi.engine.trino.event.loggers")
+      .doc("A comma separated list of engine history loggers, where engine/session/operation etc" +
+        " events go. We use spark logger by default.<ul>" +
+        " <li>JSON: the events will be written to the location of" +
+        s" ${ENGINE_EVENT_JSON_LOG_PATH.key}</li>" +
+        " <li>JDBC: to be done</li>" +
+        " <li>CUSTOM: to be done.</li></ul>")
+      .version("1.7.0")
+      .stringConf
+      .transform(_.toUpperCase(Locale.ROOT))
+      .toSequence()
+      .checkValue(
+        _.toSet.subsetOf(Set("JSON", "JDBC", "CUSTOM")),
+        "Unsupported event loggers")
+      .createWithDefault(Seq("JSON"))
 }
diff --git a/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventHandlerRegister.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventHandlerRegister.scala
index f16a48964..6c7e0893f 100644
--- a/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventHandlerRegister.scala
+++ b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventHandlerRegister.scala
@@ -18,19 +18,15 @@ package org.apache.kyuubi.events
 
 import org.apache.kyuubi.{KyuubiException, Logging}
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_LOGGERS, SERVER_EVENT_LOGGERS}
 import org.apache.kyuubi.events.EventLoggerType.EventLoggerType
 import org.apache.kyuubi.events.handler.{EventHandler, EventHandlerLoader}
 
 trait EventHandlerRegister extends Logging {
 
-  def registerEngineEventLoggers(conf: KyuubiConf): Unit = {
-    val loggers = conf.get(ENGINE_EVENT_LOGGERS)
-    register(loggers, conf)
-  }
+  protected def getLoggers(conf: KyuubiConf): Seq[String]
 
-  def registerServerEventLoggers(conf: KyuubiConf): Unit = {
-    val loggers = conf.get(SERVER_EVENT_LOGGERS)
+  def registerEventLoggers(conf: KyuubiConf): Unit = {
+    val loggers = getLoggers(conf)
     register(loggers, conf)
   }
 
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/ServerEventHandlerRegister.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/ServerEventHandlerRegister.scala
index 556a9bdb8..4ddee48dd 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/events/ServerEventHandlerRegister.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/events/ServerEventHandlerRegister.scala
@@ -19,7 +19,7 @@ package org.apache.kyuubi.events
 import java.net.InetAddress
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.config.KyuubiConf.SERVER_EVENT_JSON_LOG_PATH
+import org.apache.kyuubi.config.KyuubiConf.{SERVER_EVENT_JSON_LOG_PATH, SERVER_EVENT_LOGGERS}
 import org.apache.kyuubi.events.handler.{EventHandler, ServerJsonLoggingEventHandler}
 import org.apache.kyuubi.util.KyuubiHadoopUtils
 
@@ -35,4 +35,8 @@ object ServerEventHandlerRegister extends EventHandlerRegister {
       hadoopConf,
       kyuubiConf)
   }
+
+  override protected def getLoggers(conf: KyuubiConf): Seq[String] = {
+    conf.get(SERVER_EVENT_LOGGERS)
+  }
 }
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 4ce4f6b26..3bd791f07 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
@@ -148,7 +148,7 @@ class KyuubiServer(name: String) extends Serverable(name) {
   }
 
   private def initLoggerEventHandler(conf: KyuubiConf): Unit = {
-    ServerEventHandlerRegister.registerServerEventLoggers(conf)
+    ServerEventHandlerRegister.registerEventLoggers(conf)
   }
 
   override protected def stopServer(): Unit = {}
diff --git a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/handler/ServerJsonLoggingEventHandlerSuite.scala b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/handler/ServerJsonLoggingEventHandlerSuite.scala
index 62a2742c7..8022b6cf3 100644
--- a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/handler/ServerJsonLoggingEventHandlerSuite.scala
+++ b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/handler/ServerJsonLoggingEventHandlerSuite.scala
@@ -49,7 +49,7 @@ class ServerJsonLoggingEventHandlerSuite extends WithKyuubiServer with HiveJDBCT
     KyuubiConf()
       .set(KyuubiConf.SERVER_EVENT_LOGGERS, Seq("JSON"))
       .set(KyuubiConf.SERVER_EVENT_JSON_LOG_PATH, serverLogRoot)
-      .set(KyuubiConf.ENGINE_EVENT_LOGGERS, Seq("JSON"))
+      .set(KyuubiConf.ENGINE_SPARK_EVENT_LOGGERS, Seq("JSON"))
       .set(KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH, engineLogRoot)
   }