You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by GitBox <gi...@apache.org> on 2021/08/02 09:05:26 UTC

[GitHub] [incubator-kyuubi] yaooqinn opened a new pull request #885: Add Event for Engine

yaooqinn opened a new pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885


   <!--
   Thanks for sending a pull request!
   
   Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://kyuubi.readthedocs.io/en/latest/community/contributions.html
     2. If the PR is related to an issue in https://github.com/apache/incubator-kyuubi/issues, add '[KYUUBI #XXXX]' in your PR title, e.g., '[KYUUBI #XXXX] Your PR title ...'.
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][KYUUBI #XXXX] Your PR title ...'.
   -->
   
   ### _Why are the changes needed?_
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you add a feature, you can talk about the use case of it.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### _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.readthedocs.io/en/latest/tools/testing.html#running-tests) locally before make a pull request
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r680988322



##########
File path: externals/kyuubi-spark-monitor/pom.xml
##########
@@ -32,6 +32,12 @@
     <name>Kyuubi Project Spark Monitor</name>
 
     <dependencies>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>

Review comment:
       `kyuubi-common` depends on `scala-library`, seems no require to declare again?




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681458857



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,
+    connectionUrl: String,
+    master: String,
+    deployMode: String,
+    sparkVersion: String,
+    webUrl: String,
+    driverCores: Int,
+    driverMemoryMB: Int,
+    executorCores: Int,
+    executorMemoryMB: Int,
+    maxExecutors: Int,

Review comment:
       it's now in setttings




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1983171) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.35%`.
   > The diff coverage is `83.57%`.
   
   > :exclamation: Current head 1983171 differs from pull request most recent head 6a7ac34. Consider uploading reports for the commit 6a7ac34 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.14%   +0.35%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5346     +159     
     Branches        644      649       +5     
   ============================================
   + Hits           4087     4231     +144     
   - Misses          747      752       +5     
   - Partials        353      363      +10     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/scala/org/apache/kyuubi/sql/KyuubiAnalysis.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlBbmFseXNpcy5zY2FsYQ==) | `90.10% <ø> (ø)` | |
   | [...g/apache/kyuubi/sql/KyuubiEnsureRequirements.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlFbnN1cmVSZXF1aXJlbWVudHMuc2NhbGE=) | `30.76% <ø> (ø)` | |
   | [...pache/kyuubi/sql/KyuubiQueryStagePreparation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlRdWVyeVN0YWdlUHJlcGFyYXRpb24uc2NhbGE=) | `79.41% <ø> (ø)` | |
   | [...in/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlTUUxDb25mLnNjYWxh) | `100.00% <ø> (ø)` | |
   | [...rg/apache/kyuubi/sql/KyuubiSparkSQLExtension.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlTcGFya1NRTEV4dGVuc2lvbi5zY2FsYQ==) | `100.00% <ø> (ø)` | |
   | [...main/java/org/apache/kyuubi/jdbc/KyuubiDriver.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpRHJpdmVyLmphdmE=) | `57.14% <57.14%> (ø)` | |
   | [.../java/org/apache/kyuubi/jdbc/KyuubiConnection.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpQ29ubmVjdGlvbi5qYXZh) | `69.23% <69.23%> (ø)` | |
   | [...org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpRGF0YWJhc2VNZXRhRGF0YS5qYXZh) | `70.83% <70.83%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `73.97% <72.72%> (+2.09%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `82.60% <82.60%> (ø)` | |
   | ... and [17 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...6a7ac34](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] pan3793 commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
pan3793 commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681007454



##########
File path: externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/EventLoggingServiceSuite.scala
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.events
+
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Paths}
+
+import org.apache.kyuubi.Utils
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.engine.spark.WithSparkSQLEngine
+import org.apache.kyuubi.operation.JDBCTestUtils
+
+class EventLoggingServiceSuite extends WithSparkSQLEngine with JDBCTestUtils {
+  private val logRoot = Utils.createTempDir()
+  override def withKyuubiConf: Map[String, String] = Map(
+    KyuubiConf.ENGINE_EVENT_LOGGERS.key -> EventLoggerType.JSON.toString,
+    KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH.key -> logRoot.toString
+  )
+
+  override protected def jdbcUrl: String = getJdbcUrl
+
+
+  test("round-trip for event logging service") {
+    val engineEventPath = Paths.get(logRoot.toString, "engine", engine.engineId + ".json")
+    val reader = Files.newBufferedReader(engineEventPath, StandardCharsets.UTF_8)
+
+    val readEvent = JsonProtocol.jsonToEvent(reader.readLine())
+    assert(readEvent.isInstanceOf[KyuubiEvent])
+
+    withJdbcStatement() { statement =>
+      val table = engineEventPath.getParent.toString
+      val resultSet = statement.executeQuery(s"SELECT * FROM `json`.`${table}`")

Review comment:
       Amazing test approach!

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
##########
@@ -678,4 +678,28 @@ object KyuubiConf {
       .version("1.3.0")
       .booleanConf
       .createWithDefault(false)
+
+  val ENGINE_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
+    buildConf("engine.event.json.log.path")
+      .doc("The location of all the engine events go for the builtin JSON logger")
+      .version("1.3.0")
+      .stringConf
+      .createWithDefault("/tmp/events")

Review comment:
       `"/tmp/events"` => `"/tmp/kyuubi/events"`

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
##########
@@ -678,4 +678,30 @@ object KyuubiConf {
       .version("1.3.0")
       .booleanConf
       .createWithDefault(false)
+
+  val ENGINE_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
+    buildConf("engine.event.json.log.path")
+      .doc("The location of all the engine events go for the builtin JSON logger")
+      .version("1.3.0")
+      .stringConf
+      .createWithDefault("/tmp/kyuubi/events")
+
+  private def isValidEventLoggers(v: Set[String]): Boolean = {
+    v subsetOf(Set("SPARK", "JSON", "JDBC", "CUSTOM"))

Review comment:
       extract `Set("SPARK", "JSON", "JDBC", "CUSTOM")` to a `val`

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,
+    connectionUrl: String,
+    master: String,
+    deployMode: String,
+    sparkVersion: String,
+    webUrl: String,
+    driverCores: Int,
+    driverMemoryMB: Int,
+    executorCores: Int,
+    executorMemoryMB: Int,
+    maxExecutors: Int,

Review comment:
       Can we also add `minExecutors`? For long run ad-hoc scenario, it would be helpful for keep `minExecutors` alive all time.

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */

Review comment:
       missing `@param settings`

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EventLoggingService.scala
##########
@@ -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.engine.spark.events
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.kyuubi.SparkContextHelper
+
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.engine.spark.events.EventLoggingService._service
+import org.apache.kyuubi.service.CompositeService
+
+class EventLoggingService(engine: SparkSQLEngine)
+  extends CompositeService("EventLogging") {
+
+  private val eventLoggers = new ArrayBuffer[EventLogger]()
+
+  def onEvent(event: KyuubiEvent): Unit = {
+    eventLoggers.foreach(_.logEvent(event))
+  }
+
+  override def initialize(conf: KyuubiConf): Unit = {
+    conf.get(KyuubiConf.ENGINE_EVENT_LOGGERS)
+      .map(EventLoggerType.withName)
+      .foreach {
+        case EventLoggerType.SPARK =>
+          eventLoggers += SparkContextHelper.createSparkHistoryLogger
+        case EventLoggerType.JSON =>
+          val jsonEventLogger = new JsonEventLogger(engine.engineId)
+          addService(jsonEventLogger)
+          eventLoggers += jsonEventLogger
+        case _ => // TODO: Add more implementations

Review comment:
       Can we log something here?




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (2c141a0) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.33%`.
   > The diff coverage is `87.93%`.
   
   > :exclamation: Current head 2c141a0 differs from pull request most recent head 41baf0e. Consider uploading reports for the commit 41baf0e to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.13%   +0.33%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5357     +170     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4239     +152     
   - Misses          747      757      +10     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.66% <75.00%> (+2.79%)` | :arrow_up: |
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `80.00% <80.00%> (ø)` | |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `82.60% <82.60%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [...la/org/apache/kyuubi/service/FrontendService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2aWNlL0Zyb250ZW5kU2VydmljZS5zY2FsYQ==) | `97.00% <0.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmtfMy4xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlTUUxDb25mLnNjYWxh) | | |
   | ... and [21 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...41baf0e](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681396398



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -29,35 +29,55 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.Utils._
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.countDownLatch
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EventLoggingService}
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, RetryPolicies, ServiceDiscovery}
-import org.apache.kyuubi.service.{Serverable, Service}
+import org.apache.kyuubi.service.{Serverable, Service, ServiceState}
 import org.apache.kyuubi.util.SignalRegister
 
 case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngine") {
+
+  lazy val engineStatus: EngineEvent = EngineEvent(this)
+
+  private val eventLogging = new EventLoggingService(this)
   override val backendService = new SparkSQLBackendService(spark)
+  override val discoveryService: Service = new EngineServiceDiscovery(this)
+
   override protected def supportsServiceDiscovery: Boolean = {
     ServiceDiscovery.supportServiceDiscovery(conf)
   }
 
-  override val discoveryService: Service = new EngineServiceDiscovery(this)
-
   override def initialize(conf: KyuubiConf): Unit = {
     val listener = new SparkSQLEngineListener(this)
     spark.sparkContext.addSparkListener(listener)
+    addService(eventLogging)
     super.initialize(conf)
+    eventLogging.onEvent(engineStatus.setState(ServiceState.INITIALIZED))
   }
 
   override def start(): Unit = {
     super.start()
     // Start engine self-terminating checker after all services are ready and it can be reached by
     // all servers in engine spaces.
     backendService.sessionManager.startTerminatingChecker()
+    eventLogging.onEvent(engineStatus.setState(ServiceState.STARTED))
+  }
+
+  override def stop(): Unit = {
+    engineStatus.setState(ServiceState.STOPPED)
+    engineStatus.setEndTime(System.currentTimeMillis())
+    eventLogging.onEvent(engineStatus)
+    super.stop()
   }
 
   override protected def stopServer(): Unit = {
     countDownLatch.countDown()
   }
+
+  def engineId: String = {
+    (Seq(spark.sparkContext.applicationId) ++ spark.sparkContext.applicationAttemptId)
+      .mkString("_")

Review comment:
       changed to spark.sparkContext.applicationAttemptId.getOrElse(spark.sparkContext.applicationId)




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681411146



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,

Review comment:
       moved to settings




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3aae841) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.22%`.
   > The diff coverage is `85.05%`.
   
   > :exclamation: Current head 3aae841 differs from pull request most recent head d1b0823. Consider uploading reports for the commit d1b0823 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.01%   +0.22%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5357     +170     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4233     +146     
   - Misses          747      763      +16     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.66% <75.00%> (+2.79%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `78.26% <78.26%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [...la/org/apache/kyuubi/service/FrontendService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2aWNlL0Zyb250ZW5kU2VydmljZS5zY2FsYQ==) | `97.00% <0.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9qZGJjL0t5dXViaURyaXZlci5zY2FsYQ==) | | |
   | ... and [21 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...d1b0823](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3aae841) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.22%`.
   > The diff coverage is `85.05%`.
   
   > :exclamation: Current head 3aae841 differs from pull request most recent head 2c141a0. Consider uploading reports for the commit 2c141a0 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.01%   +0.22%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5357     +170     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4233     +146     
   - Misses          747      763      +16     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.66% <75.00%> (+2.79%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `78.26% <78.26%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [...la/org/apache/kyuubi/service/FrontendService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2aWNlL0Zyb250ZW5kU2VydmljZS5zY2FsYQ==) | `97.00% <0.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9qZGJjL0t5dXViaURyaXZlci5zY2FsYQ==) | | |
   | ... and [21 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...2c141a0](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681397074



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/JsonEventLogger.scala
##########
@@ -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.
+ */
+
+package org.apache.kyuubi.engine.spark.events
+
+import java.io.{IOException, PrintWriter}
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Path, Paths}
+import java.nio.file.attribute.PosixFilePermissions
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.service.AbstractService
+
+/**
+ * This event logger logs Kyuubi engine events in JSON file format.
+ * The hierarchical directory structure is {ENGINE_EVENT_JSON_LOG_PATH}/{eventType}/{logName}.json
+ * The {eventType} is based on core concepts of the Kyuubi systems, e.g. engine/session/statement
+ * @param logName the engine id formed of appId + attemptId(if any)
+ */
+class JsonEventLogger(logName: String)
+  extends AbstractService("JsonEventLogger") with EventLogger {
+
+  private var logRoot: Path = _
+  private val writers = new ConcurrentHashMap[String, PrintWriter]()
+
+  private def getOrUpdate(event: KyuubiEvent): PrintWriter = {
+    val writer = writers.get(event.eventType)
+    if (writer == null) {
+      val eventDir = Files.createDirectories(Paths.get(logRoot.toString, event.eventType))
+      val eventPath = Files.createFile(Paths.get(eventDir.toString, logName +  ".json"))
+
+      // TODO: make it support Hadoop compatible filesystems
+      val newWriter = new PrintWriter(Files.newBufferedWriter(eventPath, StandardCharsets.UTF_8))
+      Files.setPosixFilePermissions(eventPath, PosixFilePermissions.fromString("rwxr--r--"))
+      writers.put(event.eventType, newWriter)
+      newWriter
+    } else {
+      writer
+    }
+  }
+
+  override def initialize(conf: KyuubiConf): Unit = {
+    logRoot = Paths.get(conf.get(KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH)).toAbsolutePath
+    Files.setPosixFilePermissions(logRoot, PosixFilePermissions.fromString("rwxrwxr--"))
+    super.initialize(conf)
+  }
+
+  override def stop(): Unit = {
+    writers.values().forEach { writer => try {
+      writer.close()
+    } catch { case _: IOException => } }
+    super.stop()
+  }
+
+  override def logEvent(kyuubiEvent: KyuubiEvent): Unit = kyuubiEvent match {
+    case e: EngineEvent =>
+      val writer = getOrUpdate(e)
+      // scalastyle:off println
+      writer.println(e.toJson)
+      // scalastyle:on println
+      writer.flush()

Review comment:
       we can add a flush flag if needed later




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (1983171) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.35%`.
   > The diff coverage is `83.57%`.
   
   > :exclamation: Current head 1983171 differs from pull request most recent head 41baf0e. Consider uploading reports for the commit 41baf0e to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.14%   +0.35%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5346     +159     
     Branches        644      649       +5     
   ============================================
   + Hits           4087     4231     +144     
   - Misses          747      752       +5     
   - Partials        353      363      +10     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...n/scala/org/apache/kyuubi/sql/KyuubiAnalysis.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlBbmFseXNpcy5zY2FsYQ==) | `90.10% <ø> (ø)` | |
   | [...g/apache/kyuubi/sql/KyuubiEnsureRequirements.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlFbnN1cmVSZXF1aXJlbWVudHMuc2NhbGE=) | `30.76% <ø> (ø)` | |
   | [...pache/kyuubi/sql/KyuubiQueryStagePreparation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlRdWVyeVN0YWdlUHJlcGFyYXRpb24uc2NhbGE=) | `79.41% <ø> (ø)` | |
   | [...in/scala/org/apache/kyuubi/sql/KyuubiSQLConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlTUUxDb25mLnNjYWxh) | `100.00% <ø> (ø)` | |
   | [...rg/apache/kyuubi/sql/KyuubiSparkSQLExtension.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmstMy0xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlTcGFya1NRTEV4dGVuc2lvbi5zY2FsYQ==) | `100.00% <ø> (ø)` | |
   | [...main/java/org/apache/kyuubi/jdbc/KyuubiDriver.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpRHJpdmVyLmphdmE=) | `57.14% <57.14%> (ø)` | |
   | [.../java/org/apache/kyuubi/jdbc/KyuubiConnection.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpQ29ubmVjdGlvbi5qYXZh) | `69.23% <69.23%> (ø)` | |
   | [...org/apache/kyuubi/jdbc/KyuubiDatabaseMetaData.java](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUva3l1dWJpL2pkYmMvS3l1dWJpRGF0YWJhc2VNZXRhRGF0YS5qYXZh) | `70.83% <70.83%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `73.97% <72.72%> (+2.09%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `82.60% <82.60%> (ø)` | |
   | ... and [17 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...41baf0e](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (257e50f) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.20%`.
   > The diff coverage is `84.97%`.
   
   > :exclamation: Current head 257e50f differs from pull request most recent head 3aae841. Consider uploading reports for the commit 3aae841 to get more accurate results
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   78.99%   +0.20%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5356     +169     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4231     +144     
   - Misses          747      764      +17     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.32% <73.91%> (+2.44%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `78.26% <78.26%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [.../scala/org/apache/kyuubi/server/KyuubiServer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvS3l1dWJpU2VydmVyLnNjYWxh) | `47.36% <0.00%> (-2.64%)` | :arrow_down: |
   | [...pache/kyuubi/sql/KyuubiQueryStagePreparation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmtfMy4xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlRdWVyeVN0YWdlUHJlcGFyYXRpb24uc2NhbGE=) | `79.41% <0.00%> (-0.99%)` | :arrow_down: |
   | ... and [13 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...3aae841](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6a7ac34) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.35%`.
   > The diff coverage is `88.34%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.14%   +0.35%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5346     +159     
     Branches        644      649       +5     
   ============================================
   + Hits           4087     4231     +144     
   - Misses          747      752       +5     
   - Partials        353      363      +10     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `73.97% <72.72%> (+2.09%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `82.60% <82.60%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `90.00% <90.00%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `90.00% <90.00%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.41% <100.00%> (+0.26%)` | :arrow_up: |
   | [.../scala/org/apache/kyuubi/server/KyuubiServer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvS3l1dWJpU2VydmVyLnNjYWxh) | `47.36% <0.00%> (-2.64%)` | :arrow_down: |
   | [...la/org/apache/kyuubi/service/FrontendService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2aWNlL0Zyb250ZW5kU2VydmljZS5zY2FsYQ==) | `97.00% <0.00%> (ø)` | |
   | ... and [23 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...6a7ac34](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681369619



##########
File path: externals/kyuubi-spark-monitor/pom.xml
##########
@@ -32,6 +32,12 @@
     <name>Kyuubi Project Spark Monitor</name>
 
     <dependencies>
+
+        <dependency>
+            <groupId>org.scala-lang</groupId>
+            <artifactId>scala-library</artifactId>

Review comment:
       removed




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter edited a comment on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (3aae841) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.22%`.
   > The diff coverage is `85.05%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   79.01%   +0.22%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5357     +170     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4233     +146     
   - Misses          747      763      +16     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.66% <75.00%> (+2.79%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `78.26% <78.26%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [...la/org/apache/kyuubi/service/FrontendService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2aWNlL0Zyb250ZW5kU2VydmljZS5zY2FsYQ==) | `97.00% <0.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/jdbc/KyuubiDriver.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWhpdmUtamRiYy9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9qZGJjL0t5dXViaURyaXZlci5zY2FsYQ==) | | |
   | ... and [21 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...3aae841](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-891621000


   thanks, merged to master


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] codecov-commenter commented on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-890920271


   # [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#885](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (257e50f) into [master](https://codecov.io/gh/apache/incubator-kyuubi/commit/6c3b722174e0d046ee7701b27249fcf3f4267a92?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (6c3b722) will **increase** coverage by `0.20%`.
   > The diff coverage is `84.97%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/graphs/tree.svg?width=650&height=150&src=pr&token=925D4tb9AH&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master     #885      +/-   ##
   ============================================
   + Coverage     78.79%   78.99%   +0.20%     
   - Complexity       10       11       +1     
   ============================================
     Files           137      143       +6     
     Lines          5187     5356     +169     
     Branches        644      651       +7     
   ============================================
   + Hits           4087     4231     +144     
   - Misses          747      764      +17     
   - Partials        353      361       +8     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...a/org/apache/spark/kyuubi/SparkContextHelper.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUvc3Bhcmsva3l1dWJpL1NwYXJrQ29udGV4dEhlbHBlci5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `74.32% <73.91%> (+2.44%)` | :arrow_up: |
   | [...uubi/engine/spark/events/EventLoggingService.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnaW5nU2VydmljZS5zY2FsYQ==) | `78.26% <78.26%> (ø)` | |
   | [...pache/kyuubi/engine/spark/events/EngineEvent.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRW5naW5lRXZlbnQuc2NhbGE=) | `89.18% <89.18%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/JsonEventLogger.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvbkV2ZW50TG9nZ2VyLnNjYWxh) | `92.30% <92.30%> (ø)` | |
   | [...e/kyuubi/engine/spark/events/EventLoggerType.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvRXZlbnRMb2dnZXJUeXBlLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...ache/kyuubi/engine/spark/events/JsonProtocol.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9ldmVudHMvSnNvblByb3RvY29sLnNjYWxh) | `100.00% <100.00%> (ø)` | |
   | [...in/scala/org/apache/kyuubi/config/KyuubiConf.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLWNvbW1vbi9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9jb25maWcvS3l1dWJpQ29uZi5zY2FsYQ==) | `94.38% <100.00%> (+0.23%)` | :arrow_up: |
   | [.../scala/org/apache/kyuubi/server/KyuubiServer.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-a3l1dWJpLXNlcnZlci9zcmMvbWFpbi9zY2FsYS9vcmcvYXBhY2hlL2t5dXViaS9zZXJ2ZXIvS3l1dWJpU2VydmVyLnNjYWxh) | `47.36% <0.00%> (-2.64%)` | :arrow_down: |
   | [...pache/kyuubi/sql/KyuubiQueryStagePreparation.scala](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-ZGV2L2t5dXViaS1leHRlbnNpb24tc3BhcmtfMy4xL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL3NxbC9LeXV1YmlRdWVyeVN0YWdlUHJlcGFyYXRpb24uc2NhbGE=) | `79.41% <0.00%> (-0.99%)` | :arrow_down: |
   | ... and [13 more](https://codecov.io/gh/apache/incubator-kyuubi/pull/885/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [6c3b722...257e50f](https://codecov.io/gh/apache/incubator-kyuubi/pull/885?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] ulysses-you commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
ulysses-you commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681376304



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/JsonEventLogger.scala
##########
@@ -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.
+ */
+
+package org.apache.kyuubi.engine.spark.events
+
+import java.io.{IOException, PrintWriter}
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Path, Paths}
+import java.nio.file.attribute.PosixFilePermissions
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.service.AbstractService
+
+/**
+ * This event logger logs Kyuubi engine events in JSON file format.
+ * The hierarchical directory structure is {ENGINE_EVENT_JSON_LOG_PATH}/{eventType}/{logName}.json
+ * The {eventType} is based on core concepts of the Kyuubi systems, e.g. engine/session/statement
+ * @param logName the engine id formed of appId + attemptId(if any)
+ */
+class JsonEventLogger(logName: String)
+  extends AbstractService("JsonEventLogger") with EventLogger {
+
+  private var logRoot: Path = _
+  private val writers = new ConcurrentHashMap[String, PrintWriter]()
+
+  private def getOrUpdate(event: KyuubiEvent): PrintWriter = {

Review comment:
       shall we add lock ? seems logEvent method can be invoked by several thread.

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -29,35 +29,55 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.Utils._
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.countDownLatch
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EventLoggingService}
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, RetryPolicies, ServiceDiscovery}
-import org.apache.kyuubi.service.{Serverable, Service}
+import org.apache.kyuubi.service.{Serverable, Service, ServiceState}
 import org.apache.kyuubi.util.SignalRegister
 
 case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngine") {
+
+  lazy val engineStatus: EngineEvent = EngineEvent(this)
+
+  private val eventLogging = new EventLoggingService(this)
   override val backendService = new SparkSQLBackendService(spark)
+  override val discoveryService: Service = new EngineServiceDiscovery(this)
+
   override protected def supportsServiceDiscovery: Boolean = {
     ServiceDiscovery.supportServiceDiscovery(conf)
   }
 
-  override val discoveryService: Service = new EngineServiceDiscovery(this)
-
   override def initialize(conf: KyuubiConf): Unit = {
     val listener = new SparkSQLEngineListener(this)
     spark.sparkContext.addSparkListener(listener)
+    addService(eventLogging)
     super.initialize(conf)
+    eventLogging.onEvent(engineStatus.setState(ServiceState.INITIALIZED))
   }
 
   override def start(): Unit = {
     super.start()
     // Start engine self-terminating checker after all services are ready and it can be reached by
     // all servers in engine spaces.
     backendService.sessionManager.startTerminatingChecker()
+    eventLogging.onEvent(engineStatus.setState(ServiceState.STARTED))
+  }
+
+  override def stop(): Unit = {
+    engineStatus.setState(ServiceState.STOPPED)
+    engineStatus.setEndTime(System.currentTimeMillis())
+    eventLogging.onEvent(engineStatus)
+    super.stop()
   }
 
   override protected def stopServer(): Unit = {
     countDownLatch.countDown()
   }
+
+  def engineId: String = {
+    (Seq(spark.sparkContext.applicationId) ++ spark.sparkContext.applicationAttemptId)
+      .mkString("_")

Review comment:
       Seems too long if this is attempt app. How about:
   ```
   if (spark.sparkContext.applicationAttemptId.isDefined) {
     spark.sparkContext.applicationAttemptId.get
   } else {
     spark.sparkContext.applicationId
   }
   ```

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
##########
@@ -678,4 +678,28 @@ object KyuubiConf {
       .version("1.3.0")
       .booleanConf
       .createWithDefault(false)
+
+  val ENGINE_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
+    buildConf("engine.event.json.log.path")
+      .doc("The location of all the engine events go for the builtin JSON logger")
+      .version("1.3.0")
+      .stringConf
+      .createWithDefault("/tmp/events")

Review comment:
       `/tmp/kyuubi/events`

##########
File path: kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
##########
@@ -678,4 +678,28 @@ object KyuubiConf {
       .version("1.3.0")
       .booleanConf
       .createWithDefault(false)
+
+  val ENGINE_EVENT_JSON_LOG_PATH: ConfigEntry[String] =
+    buildConf("engine.event.json.log.path")
+      .doc("The location of all the engine events go for the builtin JSON logger")
+      .version("1.3.0")
+      .stringConf
+      .createWithDefault("/tmp/events")
+
+  val ENGINE_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
+    buildConf("engine.event.loggers")
+      .doc("A comma separated list of engine history loggers, where engine/session/operation etc" +
+        " events go.<ul>" +
+        " <li>SPARK: the events will be written to the spark history events</li>" +
+        s" <li>JSON: the events will be written to the location of" +
+        s" ${ENGINE_EVENT_JSON_LOG_PATH.key}</li>" +
+        s" <li>JDBC: to be done</li>" +
+        s" <li>CUSTOM: to be done.</li></ul>")
+      .version("1.3.0")
+      .stringConf
+      .transform(_.toUpperCase(Locale.ROOT))
+      .toSequence
+      .createWithDefault(Nil)

Review comment:
       we can add check value to ensure the input logger works

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EventLoggingService.scala
##########
@@ -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.engine.spark.events
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.kyuubi.SparkContextHelper
+
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.engine.spark.events.EventLoggingService._service
+import org.apache.kyuubi.service.CompositeService
+
+class EventLoggingService(engine: SparkSQLEngine)
+  extends CompositeService("EventLogging") {
+
+  private val eventLoggers = new ArrayBuffer[EventLogger]()
+
+  def onEvent(event: KyuubiEvent): Unit = {
+    eventLoggers.foreach(_.logEvent(event))
+  }
+
+  override def initialize(conf: KyuubiConf): Unit = {
+    conf.get(KyuubiConf.ENGINE_EVENT_LOGGERS)
+      .map(EventLoggerType.withName)
+      .foreach {
+        case EventLoggerType.SPARK =>
+          eventLoggers += SparkContextHelper.createSparkHistoryLogger
+        case EventLoggerType.JSON =>
+          val jsonEventLogger = new JsonEventLogger(engine.engineId)
+          addService(jsonEventLogger)
+          eventLoggers += jsonEventLogger
+        case _ => // TODO: Add more implementations

Review comment:
       we can throw exception here.

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,

Review comment:
       shall we record `subDomain` ?

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,
+    connectionUrl: String,
+    master: String,
+    deployMode: String,
+    sparkVersion: String,
+    webUrl: String,
+    driverCores: Int,
+    driverMemoryMB: Int,
+    executorCores: Int,
+    executorMemoryMB: Int,
+    maxExecutors: Int,
+    startTime: Long,
+    var endTime: Long = -1L,

Review comment:
       if we use `copy`, we don't need `var` anymore

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -29,35 +29,55 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.Utils._
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.countDownLatch
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EventLoggingService}
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, RetryPolicies, ServiceDiscovery}
-import org.apache.kyuubi.service.{Serverable, Service}
+import org.apache.kyuubi.service.{Serverable, Service, ServiceState}
 import org.apache.kyuubi.util.SignalRegister
 
 case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngine") {
+
+  lazy val engineStatus: EngineEvent = EngineEvent(this)
+
+  private val eventLogging = new EventLoggingService(this)
   override val backendService = new SparkSQLBackendService(spark)
+  override val discoveryService: Service = new EngineServiceDiscovery(this)
+
   override protected def supportsServiceDiscovery: Boolean = {
     ServiceDiscovery.supportServiceDiscovery(conf)
   }
 
-  override val discoveryService: Service = new EngineServiceDiscovery(this)
-
   override def initialize(conf: KyuubiConf): Unit = {
     val listener = new SparkSQLEngineListener(this)
     spark.sparkContext.addSparkListener(listener)
+    addService(eventLogging)
     super.initialize(conf)
+    eventLogging.onEvent(engineStatus.setState(ServiceState.INITIALIZED))

Review comment:
       `engineStatus.copy` ? it's not always safe if we use the same instance. 

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/JsonEventLogger.scala
##########
@@ -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.
+ */
+
+package org.apache.kyuubi.engine.spark.events
+
+import java.io.{IOException, PrintWriter}
+import java.nio.charset.StandardCharsets
+import java.nio.file.{Files, Path, Paths}
+import java.nio.file.attribute.PosixFilePermissions
+import java.util.concurrent.ConcurrentHashMap
+
+import org.apache.kyuubi.config.KyuubiConf
+import org.apache.kyuubi.service.AbstractService
+
+/**
+ * This event logger logs Kyuubi engine events in JSON file format.
+ * The hierarchical directory structure is {ENGINE_EVENT_JSON_LOG_PATH}/{eventType}/{logName}.json
+ * The {eventType} is based on core concepts of the Kyuubi systems, e.g. engine/session/statement
+ * @param logName the engine id formed of appId + attemptId(if any)
+ */
+class JsonEventLogger(logName: String)
+  extends AbstractService("JsonEventLogger") with EventLogger {
+
+  private var logRoot: Path = _
+  private val writers = new ConcurrentHashMap[String, PrintWriter]()
+
+  private def getOrUpdate(event: KyuubiEvent): PrintWriter = {
+    val writer = writers.get(event.eventType)
+    if (writer == null) {
+      val eventDir = Files.createDirectories(Paths.get(logRoot.toString, event.eventType))
+      val eventPath = Files.createFile(Paths.get(eventDir.toString, logName +  ".json"))
+
+      // TODO: make it support Hadoop compatible filesystems
+      val newWriter = new PrintWriter(Files.newBufferedWriter(eventPath, StandardCharsets.UTF_8))
+      Files.setPosixFilePermissions(eventPath, PosixFilePermissions.fromString("rwxr--r--"))
+      writers.put(event.eventType, newWriter)
+      newWriter
+    } else {
+      writer
+    }
+  }
+
+  override def initialize(conf: KyuubiConf): Unit = {
+    logRoot = Paths.get(conf.get(KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH)).toAbsolutePath
+    Files.setPosixFilePermissions(logRoot, PosixFilePermissions.fromString("rwxrwxr--"))
+    super.initialize(conf)
+  }
+
+  override def stop(): Unit = {
+    writers.values().forEach { writer => try {
+      writer.close()
+    } catch { case _: IOException => } }
+    super.stop()
+  }
+
+  override def logEvent(kyuubiEvent: KyuubiEvent): Unit = kyuubiEvent match {
+    case e: EngineEvent =>
+      val writer = getOrUpdate(e)
+      // scalastyle:off println
+      writer.println(e.toJson)
+      // scalastyle:on println
+      writer.flush()

Review comment:
       For `EngineEvent`, there is no big problem we flush every event. But we need consider cache if the size of other event is big.

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,
+    connectionUrl: String,
+    master: String,
+    deployMode: String,
+    sparkVersion: String,
+    webUrl: String,
+    driverCores: Int,
+    driverMemoryMB: Int,
+    executorCores: Int,
+    executorMemoryMB: Int,
+    maxExecutors: Int,
+    startTime: Long,
+    var endTime: Long = -1L,
+    var state: Int = 0,
+    var diagnostic: String = "") extends KyuubiEvent {
+
+  override def eventType: String = "engine"
+
+  override def schema: StructType = Encoders.product[EngineEvent].schema
+
+  override def toJson: String = JsonProtocol.productToJson(this)
+
+  override def toString: String = {
+    s"""
+       |    Spark application name: $applicationName
+       |          application ID:  $applicationId
+       |          application web UI: $webUrl
+       |          master: $master
+       |          deploy mode: $deployMode
+       |          version: $sparkVersion
+       |          driver: [cpu: $driverCores, mem: $driverMemoryMB MB]
+       |          executor: [cpu: $executorCores, mem: $executorMemoryMB MB, maxNum: $maxExecutors]
+       |    Start time: ${new Date(startTime)}
+       |    ${if (endTime != -1L) "End time: " + new Date(endTime) else ""}
+       |    User: $owner (shared mode: $shareLevel)
+       |    State: ${ServiceState(state)}
+       |    ${if (diagnostic.nonEmpty) "Diagnostic: " + diagnostic else ""}""".stripMargin
+  }
+
+  def setEndTime(time: Long): this.type = {
+    this.endTime = time
+    this
+  }
+
+  def setDiagnostic(diagnostic: String): this.type = {
+    this.diagnostic = diagnostic
+    this
+  }
+
+  def setState(newState: ServiceState): this.type = {
+    this.state = newState.id
+    this
+  }
+}
+
+object EngineEvent {
+
+  def apply(engine: SparkSQLEngine): EngineEvent = {
+    val sc = engine.spark.sparkContext
+    val webUrl = sc.getConf.getOption(
+      "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES")
+      .orElse(sc.uiWebUrl).getOrElse("")
+    // need to consider deploy mode and cluster to get core and mem
+    val driverCores = sc.getConf.getInt("spark.driver.cores", 0)
+    val driverMemory = sc.getConf.getSizeAsMb("spark.driver.memory", "1g").toInt
+    val executorCore = sc.getConf.getInt("spark.executor.cores", 1)
+    val executorMemory = sc.getConf.getSizeAsMb("spark.executor.memory", "1g").toInt
+    val dae = sc.getConf.getBoolean("spark.dynamicAllocation.enabled", defaultValue = false)
+    val maxExecutors = if (dae) {
+      sc.getConf.getInt("spark.dynamicAllocation.maxExecutors", Int.MaxValue)
+    } else {
+      sc.getConf.getInt("spark.executor.instances", 1)

Review comment:
       the default value is 2

##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EngineEvent.scala
##########
@@ -0,0 +1,147 @@
+/*
+ * 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.events
+
+import java.util.Date
+
+import org.apache.spark.sql.Encoders
+import org.apache.spark.sql.types.StructType
+
+import org.apache.kyuubi.config.KyuubiConf._
+import org.apache.kyuubi.engine.spark.SparkSQLEngine
+import org.apache.kyuubi.service.ServiceState
+import org.apache.kyuubi.service.ServiceState.ServiceState
+
+/**
+ *
+ * @param applicationId application id a.k.a, the unique id for engine
+ * @param applicationName the application name
+ * @param owner the application user
+ * @param shareLevel the share level for this engine
+ * @param connectionUrl the jdbc connection string
+ * @param master the master type, yarn, k8s, local etc.
+ * @param deployMode client/ cluster
+ * @param sparkVersion short version of spark distribution
+ * @param webUrl the tracking url of this engine
+ * @param driverCores driver cores specified
+ * @param driverMemoryMB driver memory specified
+ * @param executorCores executor cores specified
+ * @param executorMemoryMB driver memory specified
+ * @param maxExecutors max number of executors
+ * @param startTime start time
+ * @param endTime end time
+ * @param state the engine state
+ * @param diagnostic caught exceptions if any
+ */
+case class EngineEvent(
+    applicationId: String,
+    attemptId: Option[String],
+    applicationName: String,
+    owner: String,
+    shareLevel: String,
+    connectionUrl: String,
+    master: String,
+    deployMode: String,
+    sparkVersion: String,
+    webUrl: String,
+    driverCores: Int,
+    driverMemoryMB: Int,
+    executorCores: Int,
+    executorMemoryMB: Int,
+    maxExecutors: Int,
+    startTime: Long,
+    var endTime: Long = -1L,
+    var state: Int = 0,
+    var diagnostic: String = "") extends KyuubiEvent {
+
+  override def eventType: String = "engine"
+
+  override def schema: StructType = Encoders.product[EngineEvent].schema
+
+  override def toJson: String = JsonProtocol.productToJson(this)
+
+  override def toString: String = {
+    s"""
+       |    Spark application name: $applicationName
+       |          application ID:  $applicationId
+       |          application web UI: $webUrl
+       |          master: $master
+       |          deploy mode: $deployMode
+       |          version: $sparkVersion
+       |          driver: [cpu: $driverCores, mem: $driverMemoryMB MB]
+       |          executor: [cpu: $executorCores, mem: $executorMemoryMB MB, maxNum: $maxExecutors]
+       |    Start time: ${new Date(startTime)}
+       |    ${if (endTime != -1L) "End time: " + new Date(endTime) else ""}
+       |    User: $owner (shared mode: $shareLevel)
+       |    State: ${ServiceState(state)}
+       |    ${if (diagnostic.nonEmpty) "Diagnostic: " + diagnostic else ""}""".stripMargin
+  }
+
+  def setEndTime(time: Long): this.type = {
+    this.endTime = time
+    this
+  }
+
+  def setDiagnostic(diagnostic: String): this.type = {
+    this.diagnostic = diagnostic
+    this
+  }
+
+  def setState(newState: ServiceState): this.type = {
+    this.state = newState.id
+    this
+  }
+}
+
+object EngineEvent {
+
+  def apply(engine: SparkSQLEngine): EngineEvent = {
+    val sc = engine.spark.sparkContext
+    val webUrl = sc.getConf.getOption(
+      "spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES")
+      .orElse(sc.uiWebUrl).getOrElse("")
+    // need to consider deploy mode and cluster to get core and mem
+    val driverCores = sc.getConf.getInt("spark.driver.cores", 0)

Review comment:
       1 ? 0 seems overkill




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681396191



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -29,35 +29,55 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.Utils._
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.countDownLatch
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EventLoggingService}
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, RetryPolicies, ServiceDiscovery}
-import org.apache.kyuubi.service.{Serverable, Service}
+import org.apache.kyuubi.service.{Serverable, Service, ServiceState}
 import org.apache.kyuubi.util.SignalRegister
 
 case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngine") {
+
+  lazy val engineStatus: EngineEvent = EngineEvent(this)
+
+  private val eventLogging = new EventLoggingService(this)
   override val backendService = new SparkSQLBackendService(spark)
+  override val discoveryService: Service = new EngineServiceDiscovery(this)
+
   override protected def supportsServiceDiscovery: Boolean = {
     ServiceDiscovery.supportServiceDiscovery(conf)
   }
 
-  override val discoveryService: Service = new EngineServiceDiscovery(this)
-
   override def initialize(conf: KyuubiConf): Unit = {
     val listener = new SparkSQLEngineListener(this)
     spark.sparkContext.addSparkListener(listener)
+    addService(eventLogging)
     super.initialize(conf)
+    eventLogging.onEvent(engineStatus.setState(ServiceState.INITIALIZED))

Review comment:
       LGTM, by the way, copy is not safe too, we don't care much about thread-safety here




-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#issuecomment-891439863


   cc @zhang1002 @ulysses-you 


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn closed pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn closed pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885


   


-- 
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: commits-unsubscribe@kyuubi.apache.org

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



[GitHub] [incubator-kyuubi] yaooqinn commented on a change in pull request #885: Add Event for Engine

Posted by GitBox <gi...@apache.org>.
yaooqinn commented on a change in pull request #885:
URL: https://github.com/apache/incubator-kyuubi/pull/885#discussion_r681396191



##########
File path: externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
##########
@@ -29,35 +29,55 @@ import org.apache.kyuubi.Logging
 import org.apache.kyuubi.Utils._
 import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.countDownLatch
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EventLoggingService}
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.{EngineServiceDiscovery, RetryPolicies, ServiceDiscovery}
-import org.apache.kyuubi.service.{Serverable, Service}
+import org.apache.kyuubi.service.{Serverable, Service, ServiceState}
 import org.apache.kyuubi.util.SignalRegister
 
 case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngine") {
+
+  lazy val engineStatus: EngineEvent = EngineEvent(this)
+
+  private val eventLogging = new EventLoggingService(this)
   override val backendService = new SparkSQLBackendService(spark)
+  override val discoveryService: Service = new EngineServiceDiscovery(this)
+
   override protected def supportsServiceDiscovery: Boolean = {
     ServiceDiscovery.supportServiceDiscovery(conf)
   }
 
-  override val discoveryService: Service = new EngineServiceDiscovery(this)
-
   override def initialize(conf: KyuubiConf): Unit = {
     val listener = new SparkSQLEngineListener(this)
     spark.sparkContext.addSparkListener(listener)
+    addService(eventLogging)
     super.initialize(conf)
+    eventLogging.onEvent(engineStatus.setState(ServiceState.INITIALIZED))

Review comment:
       LGTM, by the way, copy is not safe also, we don't care much about thread-safety here




-- 
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: commits-unsubscribe@kyuubi.apache.org

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