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

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


The following commit(s) were added to refs/heads/master by this push:
     new e246b2f  [KYUUBI #2012] rename EventLoggingService and creates an 
correct directory
e246b2f is described below

commit e246b2f9d4f153c43b6a3648f51092710ffd83f2
Author: jiaoqingbo <[email protected]>
AuthorDate: Sat Mar 5 22:22:43 2022 +0800

    [KYUUBI #2012] rename EventLoggingService and creates an correct directory
    
    ### _Why are the changes needed?_
    
    Fix #2012 #1180
    
    In `EventLoggingService`, when `EventLoggerType` is set to `JSON`
    Kyuubi creates an error directory, it should not be the 
`ENGINE_EVENT_JSON_LOG_PATH` but the `SERVER_EVENT_JSON_LOG_PATH`.
    <img width="557" alt="微信图片_20220304165614" 
src="https://user-images.githubusercontent.com/14961757/156771039-e18fcdbe-8b99-4190-9b14-6953daa55e5f.png";>
    <img width="605" alt="微信图片_20220304165633" 
src="https://user-images.githubusercontent.com/14961757/156771053-defdee62-67f9-4f16-9d4a-8ef170304221.png";>
    
    rename `EventLoggingService` in `kyuubi-server` to 
`KyuubiEventLoggingService`
    rename `EventLoggingService`  in `kyuubi-spark-sql-engine` to 
`SparkEventLoggingService`
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including 
negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run 
test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests)
 locally before make a pull request
    
    Closes #2037 from jiaoqingbo/kyuubi2012.
    
    Closes #2012
    
    714e5ea5 [jiaoqingbo] fix ut failed
    10d1a151 [jiaoqingbo] execute dev/format
    cb99a458 [jiaoqingbo] [KYUUBI #2012] rename EventLoggingService and creates 
an correct directory
    
    Authored-by: jiaoqingbo <[email protected]>
    Signed-off-by: Kent Yao <[email protected]>
---
 .../main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala    | 4 ++--
 .../{EventLoggingService.scala => SparkEventLoggingService.scala}     | 3 ++-
 ...tLoggingServiceSuite.scala => SparkEventLoggingServiceSuite.scala} | 2 +-
 .../src/main/scala/org/apache/kyuubi/events/JsonEventLogger.scala     | 4 +---
 .../{EventLoggingService.scala => KyuubiEventLoggingService.scala}    | 4 ++--
 .../src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala        | 2 +-
 ...LoggingServiceSuite.scala => KyuubiEventLoggingServiceSuite.scala} | 2 +-
 7 files changed, 10 insertions(+), 11 deletions(-)

diff --git 
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
 
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
index 933cbea..31a2828 100644
--- 
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
+++ 
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala
@@ -33,7 +33,7 @@ import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf._
 import 
org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_SUBMIT_TIME_KEY
 import org.apache.kyuubi.engine.spark.SparkSQLEngine.{countDownLatch, 
currentEngine}
-import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore, 
EventLoggingService}
+import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore, 
SparkEventLoggingService}
 import org.apache.kyuubi.events.EventLogging
 import org.apache.kyuubi.ha.HighAvailabilityConf._
 import org.apache.kyuubi.ha.client.RetryPolicies
@@ -141,7 +141,7 @@ object SparkSQLEngine extends Logging {
     currentEngine = Some(new SparkSQLEngine(spark))
     currentEngine.foreach { engine =>
       // start event logging ahead so that we can capture all statuses
-      val eventLogging = new EventLoggingService(spark.sparkContext)
+      val eventLogging = new SparkEventLoggingService(spark.sparkContext)
       try {
         eventLogging.initialize(kyuubiConf)
         eventLogging.start()
diff --git 
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EventLoggingService.scala
 
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingService.scala
similarity index 93%
rename from 
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EventLoggingService.scala
rename to 
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingService.scala
index 249b8e9..04f644b 100644
--- 
a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/EventLoggingService.scala
+++ 
b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingService.scala
@@ -24,7 +24,7 @@ import org.apache.kyuubi.config.KyuubiConf
 import org.apache.kyuubi.config.KyuubiConf.{ENGINE_EVENT_JSON_LOG_PATH, 
ENGINE_EVENT_LOGGERS}
 import org.apache.kyuubi.events.{AbstractEventLoggingService, EventLoggerType, 
JsonEventLogger}
 
-class EventLoggingService(spark: SparkContext)
+class SparkEventLoggingService(spark: SparkContext)
   extends AbstractEventLoggingService {
 
   override def initialize(conf: KyuubiConf): Unit = synchronized {
@@ -38,6 +38,7 @@ class EventLoggingService(spark: SparkContext)
             spark.applicationAttemptId.getOrElse(spark.applicationId),
             ENGINE_EVENT_JSON_LOG_PATH,
             spark.hadoopConfiguration)
+          jsonEventLogger.createEventLogRootDir(conf, 
spark.hadoopConfiguration)
           addService(jsonEventLogger)
           addEventLogger(jsonEventLogger)
         case logger =>
diff --git 
a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/EventLoggingServiceSuite.scala
 
b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingServiceSuite.scala
similarity index 98%
rename from 
externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/EventLoggingServiceSuite.scala
rename to 
externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingServiceSuite.scala
index 007ff67..163f60e 100644
--- 
a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/EventLoggingServiceSuite.scala
+++ 
b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/events/SparkEventLoggingServiceSuite.scala
@@ -32,7 +32,7 @@ import org.apache.kyuubi.events.EventLoggerType._
 import org.apache.kyuubi.events.JsonProtocol
 import org.apache.kyuubi.operation.{HiveJDBCTestHelper, OperationHandle}
 
-class EventLoggingServiceSuite extends WithSparkSQLEngine with 
HiveJDBCTestHelper {
+class SparkEventLoggingServiceSuite extends WithSparkSQLEngine with 
HiveJDBCTestHelper {
 
   private val logRoot = "file://" + Utils.createTempDir().toString
   private val currentDate = 
Utils.getDateFromTimestamp(System.currentTimeMillis())
diff --git 
a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonEventLogger.scala 
b/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonEventLogger.scala
index 9537a7e..d29bfdf 100644
--- 
a/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonEventLogger.scala
+++ 
b/kyuubi-common/src/main/scala/org/apache/kyuubi/events/JsonEventLogger.scala
@@ -29,7 +29,6 @@ import org.apache.hadoop.fs.permission.FsPermission
 
 import org.apache.kyuubi.Logging
 import org.apache.kyuubi.config.{ConfigEntry, KyuubiConf}
-import org.apache.kyuubi.config.KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH
 import org.apache.kyuubi.events.JsonEventLogger._
 import org.apache.kyuubi.service.AbstractService
 
@@ -114,9 +113,8 @@ class JsonEventLogger(
     stream.foreach(_.hflush())
   }
 
-  // This method is only called by kyuubiServer
   def createEventLogRootDir(conf: KyuubiConf, hadoopConf: Configuration): Unit 
= {
-    val logRoot: URI = URI.create(conf.get(ENGINE_EVENT_JSON_LOG_PATH))
+    val logRoot: URI = URI.create(conf.get(logPath))
     val fs: FileSystem = FileSystem.get(logRoot, hadoopConf)
     val success: Boolean = FileSystem.mkdirs(fs, new Path(logRoot), 
JSON_LOG_DIR_PERM)
     if (!success) {
diff --git 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/EventLoggingService.scala
 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiEventLoggingService.scala
similarity index 93%
rename from 
kyuubi-server/src/main/scala/org/apache/kyuubi/server/EventLoggingService.scala
rename to 
kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiEventLoggingService.scala
index adb3e12..4c53b94 100644
--- 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/EventLoggingService.scala
+++ 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiEventLoggingService.scala
@@ -24,7 +24,7 @@ import 
org.apache.kyuubi.config.KyuubiConf.{SERVER_EVENT_JSON_LOG_PATH, SERVER_E
 import org.apache.kyuubi.events.{AbstractEventLoggingService, EventLoggerType, 
JsonEventLogger}
 import org.apache.kyuubi.util.KyuubiHadoopUtils
 
-class EventLoggingService extends AbstractEventLoggingService {
+class KyuubiEventLoggingService extends AbstractEventLoggingService {
 
   override def initialize(conf: KyuubiConf): Unit = {
     val hadoopConf = KyuubiHadoopUtils.newHadoopConf(conf)
@@ -37,7 +37,7 @@ class EventLoggingService extends AbstractEventLoggingService 
{
             s"server-$hostName",
             SERVER_EVENT_JSON_LOG_PATH,
             hadoopConf)
-          // TODO: #1180 kyuubiServerEvent need create logRoot automatically
+
           jsonEventLogger.createEventLogRootDir(conf, hadoopConf)
           addService(jsonEventLogger)
           addEventLogger(jsonEventLogger)
diff --git 
a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala 
b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
index 3f6e34e..8c45aea 100644
--- a/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
+++ b/kyuubi-server/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala
@@ -154,7 +154,7 @@ class KyuubiServer(name: String) extends Serverable(name) {
         throw new UnsupportedOperationException(s"Frontend protocol $other is 
not supported yet.")
     }
 
-  private val eventLoggingService = new EventLoggingService()
+  private val eventLoggingService = new KyuubiEventLoggingService()
 
   override def initialize(conf: KyuubiConf): Unit = synchronized {
     val kinit = new KinitAuxiliaryService()
diff --git 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/KyuubiEventLoggingServiceSuite.scala
similarity index 99%
rename from 
kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
rename to 
kyuubi-server/src/test/scala/org/apache/kyuubi/events/KyuubiEventLoggingServiceSuite.scala
index 370b661..25d2439 100644
--- 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
+++ 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/KyuubiEventLoggingServiceSuite.scala
@@ -34,7 +34,7 @@ import org.apache.kyuubi.operation.OperationState._
 import org.apache.kyuubi.server.KyuubiServer
 import org.apache.kyuubi.service.ServiceState
 
-class EventLoggingServiceSuite extends WithKyuubiServer with 
HiveJDBCTestHelper {
+class KyuubiEventLoggingServiceSuite extends WithKyuubiServer with 
HiveJDBCTestHelper {
 
   private val engineLogRoot = "file://" + Utils.createTempDir().toString
   private val serverLogRoot = "file://" + Utils.createTempDir().toString

Reply via email to