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

[incubator-kyuubi] branch master updated: [KYUUBI #2323] Separate events to a submodule - kyuubi-event

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

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


The following commit(s) were added to refs/heads/master by this push:
     new d851b23a6 [KYUUBI #2323] Separate events to a submodule - kyuubi-event
d851b23a6 is described below

commit d851b23a60ffd862deb8c416559a9fedd0cb3f27
Author: Min Zhao <zh...@163.com>
AuthorDate: Tue Apr 19 12:06:23 2022 +0800

    [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    
    ### _Why are the changes needed?_
    
    Separate events to a submodule - kyuubi-event
    
    ### _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 #2392 from zhaomin1423/event-module.
    
    Closes #2323
    
    7efa923f [Min Zhao] fix unit test
    07126f43 [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    cccbdf11 [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    194aca82 [Min Zhao] add shade and log4j to test/resources
    f75891a1 [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    357395fa [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    b6dfd186 [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    fc876688 [Min Zhao] [KYUUBI #2323] Separate events to a submodule - kyuubi-event
    1b86d41b [Min Zhao] seperate event
    
    Authored-by: Min Zhao <zh...@163.com>
    Signed-off-by: Kent Yao <ya...@apache.org>
---
 .github/labeler.yml                                |  3 +
 dev/kyuubi-codecov/pom.xml                         |  6 ++
 externals/kyuubi-hive-sql-engine/pom.xml           |  7 +++
 externals/kyuubi-spark-sql-engine/pom.xml          |  7 +++
 .../apache/kyuubi/session/AbstractSession.scala    |  3 -
 .../scala/org/apache/kyuubi/session/Session.scala  |  2 -
 kyuubi-events/pom.xml                              | 67 ++++++++++++++++++++++
 .../scala/org/apache/kyuubi/events/EventBus.scala  |  0
 .../org/apache/kyuubi/events/EventLoggerType.scala |  0
 .../org/apache/kyuubi/events/JsonProtocol.scala    |  0
 .../org/apache/kyuubi/events/KyuubiEvent.scala     |  0
 .../events/handler/JsonLoggingEventHandler.scala   |  0
 .../org/apache/kyuubi/events/handler/package.scala |  0
 .../src/test/resources/log4j2-test.properties      | 57 ++++++++++++++++++
 .../org/apache/kyuubi/events/EventBusSuite.scala   |  0
 kyuubi-server/pom.xml                              |  6 ++
 .../kyuubi/server/api/v1/SessionsResource.scala    |  5 +-
 .../kyuubi/session/KyuubiBatchSessionImpl.scala    |  6 +-
 .../org/apache/kyuubi/session/KyuubiSession.scala  | 22 +++----
 .../apache/kyuubi/session/KyuubiSessionImpl.scala  |  6 +-
 pom.xml                                            |  1 +
 21 files changed, 175 insertions(+), 23 deletions(-)

diff --git a/.github/labeler.yml b/.github/labeler.yml
index 9aca743e7..0e1d45a73 100644
--- a/.github/labeler.yml
+++ b/.github/labeler.yml
@@ -65,6 +65,9 @@
   - "kyuubi-hive-jdbc/**/*"
   - "kyuubi-hive-jdbc-shaded/**/*"
 
+"module:events":
+  - "kyuubi-events/**/*"
+
 "module:flink":
   - "externals/kyuubi-flink-sql-engine/**/*"
   - "integration-tests/kyuubi-flink-it/**/*"
diff --git a/dev/kyuubi-codecov/pom.xml b/dev/kyuubi-codecov/pom.xml
index 48ed37fdb..8623a985a 100644
--- a/dev/kyuubi-codecov/pom.xml
+++ b/dev/kyuubi-codecov/pom.xml
@@ -45,6 +45,12 @@
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-events_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.kyuubi</groupId>
             <artifactId>kyuubi-zookeeper_${scala.binary.version}</artifactId>
diff --git a/externals/kyuubi-hive-sql-engine/pom.xml b/externals/kyuubi-hive-sql-engine/pom.xml
index 5d328041b..2e282cbd3 100644
--- a/externals/kyuubi-hive-sql-engine/pom.xml
+++ b/externals/kyuubi-hive-sql-engine/pom.xml
@@ -40,6 +40,12 @@
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-events_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.kyuubi</groupId>
             <artifactId>kyuubi-ha_${scala.binary.version}</artifactId>
@@ -166,6 +172,7 @@
                         <include>com.fasterxml.jackson.core:jackson-databind</include>
                         <include>com.fasterxml.jackson.module:jackson-module-scala_2.12</include>
                         <include>org.apache.kyuubi:kyuubi-common_${scala.binary.version}</include>
+                        <include>org.apache.kyuubi:kyuubi-events_${scala.binary.version}</include>
                         <include>org.apache.kyuubi:kyuubi-ha_${scala.binary.version}</include>
                         <include>org.apache.curator:curator-client</include>
                         <include>org.apache.curator:curator-framework</include>
diff --git a/externals/kyuubi-spark-sql-engine/pom.xml b/externals/kyuubi-spark-sql-engine/pom.xml
index ce369b62f..548810ff3 100644
--- a/externals/kyuubi-spark-sql-engine/pom.xml
+++ b/externals/kyuubi-spark-sql-engine/pom.xml
@@ -39,6 +39,12 @@
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-events_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.kyuubi</groupId>
             <artifactId>kyuubi-ha_${scala.binary.version}</artifactId>
@@ -187,6 +193,7 @@
                     <artifactSet>
                         <includes>
                             <include>org.apache.kyuubi:kyuubi-common_${scala.binary.version}</include>
+                            <include>org.apache.kyuubi:kyuubi-events_${scala.binary.version}</include>
                             <include>org.apache.kyuubi:kyuubi-ha_${scala.binary.version}</include>
                             <include>org.apache.curator:curator-client</include>
                             <include>org.apache.curator:curator-framework</include>
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 8300795ad..f1602ca77 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
@@ -22,7 +22,6 @@ import scala.collection.JavaConverters._
 import org.apache.hive.service.rpc.thrift.{TGetInfoType, TGetInfoValue, TProtocolVersion, TRowSet, TTableSchema}
 
 import org.apache.kyuubi.{KyuubiSQLException, Logging}
-import org.apache.kyuubi.events.KyuubiEvent
 import org.apache.kyuubi.operation.{Operation, OperationHandle}
 import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation
 import org.apache.kyuubi.operation.log.OperationLog
@@ -246,6 +245,4 @@ abstract class AbstractSession(
   override def open(): Unit = {
     OperationLog.createOperationLogRootDirectory(this)
   }
-
-  override def getSessionEvent: Option[KyuubiEvent] = None
 }
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 0218b6580..f9b8c2255 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
@@ -19,7 +19,6 @@ package org.apache.kyuubi.session
 
 import org.apache.hive.service.rpc.thrift.{TGetInfoType, TGetInfoValue, TProtocolVersion, TRowSet, TTableSchema}
 
-import org.apache.kyuubi.events.KyuubiEvent
 import org.apache.kyuubi.operation.FetchOrientation.FetchOrientation
 import org.apache.kyuubi.operation.OperationHandle
 
@@ -38,7 +37,6 @@ trait Session {
   def lastAccessTime: Long
   def lastIdleTime: Long
   def getNoOperationTime: Long
-  def getSessionEvent: Option[KyuubiEvent]
 
   def sessionManager: SessionManager
 
diff --git a/kyuubi-events/pom.xml b/kyuubi-events/pom.xml
new file mode 100644
index 000000000..05712fc46
--- /dev/null
+++ b/kyuubi-events/pom.xml
@@ -0,0 +1,67 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.kyuubi</groupId>
+        <artifactId>kyuubi-parent</artifactId>
+        <version>1.6.0-SNAPSHOT</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>kyuubi-events_2.12</artifactId>
+    <name>Kyuubi Project Events</name>
+    <packaging>jar</packaging>
+    <url>https://kyuubi.apache.org/</url>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-common_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-common_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+            <type>test-jar</type>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.slf4j</groupId>
+            <artifactId>jul-to-slf4j</artifactId>
+            <scope>test</scope>
+        </dependency>
+    </dependencies>
+
+    <build>
+        <outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
+        <testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
+
+        <testResources>
+            <testResource>
+                <directory>${project.basedir}/src/test/resources</directory>
+            </testResource>
+        </testResources>
+    </build>
+
+</project>
\ No newline at end of file
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventBus.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventBus.scala
similarity index 100%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventBus.scala
rename to kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventBus.scala
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventLoggerType.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventLoggerType.scala
similarity index 100%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventLoggerType.scala
rename to kyuubi-events/src/main/scala/org/apache/kyuubi/events/EventLoggerType.scala
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala
similarity index 100%
copy from kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala
copy to kyuubi-events/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/KyuubiEvent.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/KyuubiEvent.scala
similarity index 100%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/KyuubiEvent.scala
rename to kyuubi-events/src/main/scala/org/apache/kyuubi/events/KyuubiEvent.scala
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/handler/JsonLoggingEventHandler.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/handler/JsonLoggingEventHandler.scala
similarity index 100%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/handler/JsonLoggingEventHandler.scala
rename to kyuubi-events/src/main/scala/org/apache/kyuubi/events/handler/JsonLoggingEventHandler.scala
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/handler/package.scala b/kyuubi-events/src/main/scala/org/apache/kyuubi/events/handler/package.scala
similarity index 100%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/handler/package.scala
rename to kyuubi-events/src/main/scala/org/apache/kyuubi/events/handler/package.scala
diff --git a/kyuubi-events/src/test/resources/log4j2-test.properties b/kyuubi-events/src/test/resources/log4j2-test.properties
new file mode 100644
index 000000000..3975a528b
--- /dev/null
+++ b/kyuubi-events/src/test/resources/log4j2-test.properties
@@ -0,0 +1,57 @@
+#
+# 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.
+#
+
+# Set everything to be logged to the file target/unit-tests.log
+rootLogger.level = debug
+rootLogger.appenderRef.stdout.ref = STDOUT
+rootLogger.appenderRef.file.ref = File
+
+# Console Appender
+appender.console.type = Console
+appender.console.name = STDOUT
+appender.console.target = SYSTEM_OUT
+appender.console.layout.type = PatternLayout
+appender.console.layout.pattern = %d{HH:mm:ss.SSS} %p %c: %m%n
+
+appender.console.filter.1.type = Filters
+
+appender.console.filter.1.a.type = ThresholdFilter
+appender.console.filter.1.a.level = fatal
+
+# SPARK-34128: Suppress undesirable TTransportException warnings, due to THRIFT-4805
+appender.console.filter.1.b.type = RegexFilter
+appender.console.filter.1.b.regex = .*Thrift error occurred during processing of message.*
+appender.console.filter.1.b.onMatch = deny
+appender.console.filter.1.b.onMismatch = neutral
+
+# File Appender
+appender.file.type = File
+appender.file.name = File
+appender.file.fileName = target/unit-tests.log
+appender.file.layout.type = PatternLayout
+appender.file.layout.pattern = %d{HH:mm:ss.SSS} %t %p %c{1}: %m%n
+
+appender.file.filter.1.type = Filters
+
+appender.file.filter.1.a.type = RegexFilter
+appender.file.filter.1.a.regex = .*Thrift error occurred during processing of message.*
+appender.file.filter.1.a.onMatch = deny
+appender.file.filter.1.a.onMismatch = neutral
+
+# Set the logger level of File Appender to DEBUG
+appender.file.filter.1.b.type = ThresholdFilter
+appender.file.filter.1.b.level = debug
diff --git a/kyuubi-common/src/test/scala/org/apache/kyuubi/events/EventBusSuite.scala b/kyuubi-events/src/test/scala/org/apache/kyuubi/events/EventBusSuite.scala
similarity index 100%
rename from kyuubi-common/src/test/scala/org/apache/kyuubi/events/EventBusSuite.scala
rename to kyuubi-events/src/test/scala/org/apache/kyuubi/events/EventBusSuite.scala
diff --git a/kyuubi-server/pom.xml b/kyuubi-server/pom.xml
index 15ea574e7..27d84edb1 100644
--- a/kyuubi-server/pom.xml
+++ b/kyuubi-server/pom.xml
@@ -38,6 +38,12 @@
             <version>${project.version}</version>
         </dependency>
 
+        <dependency>
+            <groupId>org.apache.kyuubi</groupId>
+            <artifactId>kyuubi-events_${scala.binary.version}</artifactId>
+            <version>${project.version}</version>
+        </dependency>
+
         <dependency>
             <groupId>org.apache.kyuubi</groupId>
             <artifactId>kyuubi-server-plugin</artifactId>
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 a0193fd85..a8acffc1b 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
@@ -32,7 +32,7 @@ import org.apache.kyuubi.events.KyuubiEvent
 import org.apache.kyuubi.operation.OperationHandle
 import org.apache.kyuubi.server.api.ApiRequestContext
 import org.apache.kyuubi.server.http.authentication.AuthenticationFilter
-import org.apache.kyuubi.session.SessionHandle
+import org.apache.kyuubi.session.{KyuubiSession, SessionHandle}
 import org.apache.kyuubi.session.SessionHandle.parseSessionHandle
 
 @Tag(name = "Session")
@@ -69,7 +69,8 @@ private[v1] class SessionsResource extends ApiRequestContext with Logging {
   @Path("{sessionHandle}")
   def sessionInfo(@PathParam("sessionHandle") sessionHandleStr: String): KyuubiEvent = {
     try {
-      fe.be.sessionManager.getSession(parseSessionHandle(sessionHandleStr)).getSessionEvent.get
+      fe.be.sessionManager.getSession(parseSessionHandle(sessionHandleStr))
+        .asInstanceOf[KyuubiSession].getSessionEvent.get
     } catch {
       case NonFatal(e) =>
         error(s"Invalid $sessionHandleStr", e)
diff --git a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiBatchSessionImpl.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiBatchSessionImpl.scala
index 497c4fb88..8352cd031 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiBatchSessionImpl.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiBatchSessionImpl.scala
@@ -21,7 +21,7 @@ import com.codahale.metrics.MetricRegistry
 import org.apache.hive.service.rpc.thrift.TProtocolVersion
 
 import org.apache.kyuubi.config.KyuubiConf
-import org.apache.kyuubi.events.{EventBus, KyuubiEvent, KyuubiSessionEvent}
+import org.apache.kyuubi.events.{EventBus, KyuubiSessionEvent}
 import org.apache.kyuubi.metrics.MetricsConstants.{CONN_OPEN, CONN_TOTAL}
 import org.apache.kyuubi.metrics.MetricsSystem
 import org.apache.kyuubi.operation.OperationState
@@ -36,7 +36,7 @@ class KyuubiBatchSessionImpl(
     sessionManager: KyuubiSessionManager,
     val sessionConf: KyuubiConf,
     batchRequest: BatchRequest)
-  extends AbstractSession(protocol, user, password, ipAddress, conf, sessionManager) {
+  extends KyuubiSession(protocol, user, password, ipAddress, conf, sessionManager) {
   override val handle: SessionHandle = sessionManager.newBatchSessionHandle(protocol)
   val batchId: String = handle.identifier.toString
 
@@ -50,7 +50,7 @@ class KyuubiBatchSessionImpl(
   private val sessionEvent = KyuubiSessionEvent(this)
   EventBus.post(sessionEvent)
 
-  override def getSessionEvent: Option[KyuubiEvent] = {
+  override def getSessionEvent: Option[KyuubiSessionEvent] = {
     Option(sessionEvent)
   }
 
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSession.scala
similarity index 62%
rename from kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala
rename to kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSession.scala
index 32aef4f51..cd2a181c8 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonProtocol.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/session/KyuubiSession.scala
@@ -14,19 +14,21 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.kyuubi.session
 
-package org.apache.kyuubi.events
+import org.apache.hive.service.rpc.thrift.TProtocolVersion
 
-import com.fasterxml.jackson.databind.ObjectMapper
-import com.fasterxml.jackson.module.scala.DefaultScalaModule
+import org.apache.kyuubi.events.KyuubiSessionEvent
 
-object JsonProtocol {
+abstract class KyuubiSession(
+    protocol: TProtocolVersion,
+    user: String,
+    password: String,
+    ipAddress: String,
+    conf: Map[String, String],
+    sessionManager: KyuubiSessionManager)
+  extends AbstractSession(protocol, user, password, ipAddress, conf, sessionManager) {
 
-  private val mapper = new ObjectMapper().registerModule(DefaultScalaModule)
+  def getSessionEvent: Option[KyuubiSessionEvent]
 
-  def productToJson[T <: KyuubiEvent](value: T): String = mapper.writeValueAsString(value)
-
-  def jsonToEvent[T <: KyuubiEvent](jsonValue: String, cls: Class[T]): KyuubiEvent = {
-    mapper.readValue(jsonValue, cls)
-  }
 }
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 cec83ac1b..2d0a710d5 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
@@ -27,7 +27,7 @@ import org.apache.kyuubi.client.KyuubiSyncThriftClient
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
 import org.apache.kyuubi.engine.EngineRef
-import org.apache.kyuubi.events.{EventBus, KyuubiEvent, KyuubiSessionEvent}
+import org.apache.kyuubi.events.{EventBus, KyuubiSessionEvent}
 import org.apache.kyuubi.ha.client.DiscoveryClientProvider._
 import org.apache.kyuubi.metrics.MetricsConstants._
 import org.apache.kyuubi.metrics.MetricsSystem
@@ -43,7 +43,7 @@ class KyuubiSessionImpl(
     conf: Map[String, String],
     sessionManager: KyuubiSessionManager,
     sessionConf: KyuubiConf)
-  extends AbstractSession(protocol, user, password, ipAddress, conf, sessionManager) {
+  extends KyuubiSession(protocol, user, password, ipAddress, conf, sessionManager) {
 
   private[kyuubi] val optimizedConf: Map[String, String] = {
     val confOverlay = sessionManager.sessionConfAdvisor.getConfOverlay(
@@ -71,7 +71,7 @@ class KyuubiSessionImpl(
   private val sessionEvent = KyuubiSessionEvent(this)
   EventBus.post(sessionEvent)
 
-  override def getSessionEvent: Option[KyuubiEvent] = {
+  override def getSessionEvent: Option[KyuubiSessionEvent] = {
     Option(sessionEvent)
   }
 
diff --git a/pom.xml b/pom.xml
index c6663bf12..d32a25bfa 100644
--- a/pom.xml
+++ b/pom.xml
@@ -77,6 +77,7 @@
         <module>kyuubi-assembly</module>
         <module>kyuubi-common</module>
         <module>kyuubi-ctl</module>
+        <module>kyuubi-events</module>
         <module>kyuubi-ha</module>
         <module>kyuubi-hive-beeline</module>
         <module>kyuubi-hive-jdbc</module>