You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by GitBox <gi...@apache.org> on 2022/03/29 04:48:09 UTC

[GitHub] [incubator-kyuubi] ulysses-you commented on a change in pull request #1853: [KYUUBI #1798] Add EventBus module to unify the distribution and subscription of Kyuubi's events

ulysses-you commented on a change in pull request #1853:
URL: https://github.com/apache/incubator-kyuubi/pull/1853#discussion_r837045518



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -170,15 +169,37 @@ object SparkSQLEngine extends Logging {
           kyuubiConf)
         val event = EngineEvent(engine)
         info(event)
-        EventLogging.onEvent(event)
+        EventBus.post(event)
       } catch {
         case t: Throwable =>
           throw new KyuubiException(s"Failed to start SparkSQLEngine: ${t.getMessage}", t)
       }
       // Stop engine before SparkContext stopped to avoid calling a stopped SparkContext
       addShutdownHook(() => engine.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 2)
-      addShutdownHook(() => eventLogging.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 1)
     }
+
+    def initLoggerEventHandler(conf: KyuubiConf): Unit = {
+      conf.get(ENGINE_EVENT_LOGGERS)
+        .map(EventLoggerType.withName)
+        .foreach {
+          case EventLoggerType.SPARK =>
+            EventBus.register[KyuubiEvent](new SparkHistoryLoggingEventHandler(spark.sparkContext))
+          case EventLoggerType.JSON =>
+            val handler = SparkJsonLoggingEventHandler(
+              spark.sparkContext.applicationAttemptId.getOrElse(spark.sparkContext.applicationId),

Review comment:
       `applicationAttemptId` is a number of attempt count, e.g. `1`
   
   The right name maybe:
   ```
   spark.sparkContext.applicationAttemptId
     .map(id => s"${spark.sparkContext.applicationId}_$id")
     .getOrElse(spark.sparkContext.applicationId)
   ```

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -170,15 +169,37 @@ object SparkSQLEngine extends Logging {
           kyuubiConf)
         val event = EngineEvent(engine)
         info(event)
-        EventLogging.onEvent(event)
+        EventBus.post(event)
       } catch {
         case t: Throwable =>
           throw new KyuubiException(s"Failed to start SparkSQLEngine: ${t.getMessage}", t)
       }
       // Stop engine before SparkContext stopped to avoid calling a stopped SparkContext
       addShutdownHook(() => engine.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 2)
-      addShutdownHook(() => eventLogging.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 1)
     }
+
+    def initLoggerEventHandler(conf: KyuubiConf): Unit = {
+      conf.get(ENGINE_EVENT_LOGGERS)
+        .map(EventLoggerType.withName)
+        .foreach {
+          case EventLoggerType.SPARK =>
+            EventBus.register[KyuubiEvent](new SparkHistoryLoggingEventHandler(spark.sparkContext))
+          case EventLoggerType.JSON =>
+            val handler = SparkJsonLoggingEventHandler(
+              spark.sparkContext.applicationAttemptId.getOrElse(spark.sparkContext.applicationId),

Review comment:
       I see this code is coied, just fixes it together

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventBus.scala
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.events
+
+import scala.collection.mutable
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.Future
+import scala.reflect.{classTag, ClassTag}
+import scala.util.Try
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.events.handler.EventHandler
+
+sealed trait EventBus {
+  def post[T <: KyuubiEvent](event: T): Unit
+
+  def register[T <: KyuubiEvent: ClassTag](et: EventHandler[T]): EventBus
+
+  def registerAsync[T <: KyuubiEvent: ClassTag](et: EventHandler[T]): EventBus
+}
+
+object EventBus extends Logging {
+  private val defaultEventBus = EventBusLive()
+
+  def apply(): EventBus = EventBusLive()
+
+  // Exposed api
+  def post[T <: KyuubiEvent](event: T): Unit = defaultEventBus.post[T](event)
+
+  def register[T <: KyuubiEvent: ClassTag](et: EventHandler[T]): EventBus =
+    defaultEventBus.register[T](et)
+
+  def registerAsync[T <: KyuubiEvent: ClassTag](et: EventHandler[T]): EventBus =
+    defaultEventBus.registerAsync[T](et)
+
+  private case class EventBusLive() extends EventBus {
+    private[this] val eventHandlerRegistry = new Registry
+    private[this] val asyncEventHandlerRegistry = new Registry

Review comment:
       add `lazy`

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -170,15 +169,37 @@ object SparkSQLEngine extends Logging {
           kyuubiConf)
         val event = EngineEvent(engine)
         info(event)
-        EventLogging.onEvent(event)
+        EventBus.post(event)
       } catch {
         case t: Throwable =>
           throw new KyuubiException(s"Failed to start SparkSQLEngine: ${t.getMessage}", t)
       }
       // Stop engine before SparkContext stopped to avoid calling a stopped SparkContext
       addShutdownHook(() => engine.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 2)
-      addShutdownHook(() => eventLogging.stop(), SPARK_CONTEXT_SHUTDOWN_PRIORITY + 1)
     }
+
+    def initLoggerEventHandler(conf: KyuubiConf): Unit = {
+      conf.get(ENGINE_EVENT_LOGGERS)
+        .map(EventLoggerType.withName)
+        .foreach {
+          case EventLoggerType.SPARK =>
+            EventBus.register[KyuubiEvent](new SparkHistoryLoggingEventHandler(spark.sparkContext))
+          case EventLoggerType.JSON =>
+            val handler = SparkJsonLoggingEventHandler(
+              spark.sparkContext.applicationAttemptId.getOrElse(spark.sparkContext.applicationId),
+              ENGINE_EVENT_JSON_LOG_PATH,
+              spark.sparkContext.hadoopConfiguration,
+              conf)
+
+            // register JsonLogger as a event handler for default event bus
+            EventBus.register[KyuubiEvent](handler)
+          case logger =>
+            // TODO: Add more implementations
+            throw new IllegalArgumentException(s"Unrecognized event logger: $logger")
+        }
+

Review comment:
       unnecessary line

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventBus.scala
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.events
+
+import scala.collection.mutable
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.Future
+import scala.reflect.{classTag, ClassTag}
+import scala.util.Try
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.events.handler.EventHandler
+
+sealed trait EventBus {
+  def post[T <: KyuubiEvent](event: T): Unit
+
+  def register[T <: KyuubiEvent: ClassTag](et: EventHandler[T]): EventBus

Review comment:
       why this and some other place name it `et`...  how about `eventHander` ? 

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/events/EventBus.scala
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.events
+
+import scala.collection.mutable
+import scala.concurrent.ExecutionContext.Implicits.global
+import scala.concurrent.Future
+import scala.reflect.{classTag, ClassTag}
+import scala.util.Try
+
+import org.apache.kyuubi.Logging
+import org.apache.kyuubi.events.handler.EventHandler
+
+sealed trait EventBus {

Review comment:
       can we enrich the class description ?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: notifications-unsubscribe@kyuubi.apache.org
For additional commands, e-mail: notifications-help@kyuubi.apache.org