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 354e3c0  [KYUUBI #1068] event log can't upload to hdfs
354e3c0 is described below

commit 354e3c0b5a3860253ac29cd2ea8e24edbe7d6b57
Author: 张宇翔 <[email protected]>
AuthorDate: Mon Sep 13 17:59:13 2021 +0800

    [KYUUBI #1068] event log can't upload to hdfs
    
    bug:
    #1068
    
    This version, event log don't support upload file to hdfs
    
    ### _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/develop_tools/testing.html#running-tests)
 locally before make a pull request
    
    Closes #1075 from zhang1002/branch-1.3_supportLogUploadHdfs.
    
    Closes #1068
    
    7b1483c5 [张宇翔] Modify some UT
    27aa4596 [张宇翔] Modify some UT
    c8223914 [张宇翔] Modify some UT
    8b3ef67b [张宇翔] Modify some UT
    fa06ead9 [张宇翔] Merge branch 'master' into branch-1.3_supportLogUploadHdfs
    8384a6e2 [张宇翔] Merge remote-tracking branch 'upstream/master'
    8801ece7 [张宇翔] Remove "toString" from ut
    3288724b [张宇翔] Modify some UT
    0bd7ecbd [张宇翔] Support the eventLog upload to hdfs
    714ab1b8 [张宇翔] Merge remote-tracking branch 'upstream/master'
    af28520b [张宇翔] Merge remote-tracking branch 'upstream/master'
    c328c884 [张宇翔] Merge remote-tracking branch 'upstream/master'
    abcaae1f [张宇翔] Merge remote-tracking branch 'upstream/master'
    f0a7e566 [张宇翔] Merge remote-tracking branch 'upstream/master'
    112aa6b6 [张宇翔] Merge remote-tracking branch 'upstream/master'
    c55652fd [张宇翔] Merge remote-tracking branch 'upstream/master'
    ef5e8c55 [张宇翔] Merge remote-tracking branch 'upstream/master'
    5a75384c [张宇翔] Merge remote-tracking branch 'upstream/master'
    8da4f7ed [张宇翔] Merge remote-tracking branch 'upstream/master'
    a7ba1bfc [张宇翔] Merge remote-tracking branch 'upstream/master'
    b662a989 [张宇翔] Merge remote-tracking branch 'upstream/master'
    4c8f3b87 [张宇翔] Merge remote-tracking branch 'upstream/master'
    8b686767 [张宇翔] Merge remote-tracking branch 'upstream/master'
    cf99e309 [张宇翔] Merge remote-tracking branch 'upstream/master'
    0afaa578 [张宇翔] Merge remote-tracking branch 'upstream/master'
    b24fea07 [张宇翔] Merge remote-tracking branch 'upstream/master'
    e517cfc5 [张宇翔] Merge remote-tracking branch 'upstream/master'
    18aebe76 [张宇翔] Merge remote-tracking branch 'upstream/master'
    f248bef7 [张宇翔] Merge remote-tracking branch 'upstream/master'
    5ffb54f3 [张宇翔] Add kyuubi-spark-monitor module for nightly.yml
    
    Authored-by: 张宇翔 <[email protected]>
    Signed-off-by: Kent Yao <[email protected]>
---
 docs/deployment/settings.md                        |  2 +-
 .../spark/events/EventLoggingServiceSuite.scala    | 25 +++++++++++++---------
 .../org/apache/kyuubi/config/KyuubiConf.scala      |  6 ++++--
 .../org/apache/kyuubi/events/JsonEventLogger.scala |  3 +--
 .../kyuubi/events/EventLoggingServiceSuite.scala   | 16 +++++++-------
 5 files changed, 29 insertions(+), 23 deletions(-)

diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index a64c07b..a3c3bcf 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -177,7 +177,7 @@ kyuubi\.engine<br>\.deregister\.exception<br>\.classes|<div 
style='width: 65pt;w
 kyuubi\.engine<br>\.deregister\.exception<br>\.messages|<div style='width: 
65pt;word-wrap: break-word;white-space: normal'></div>|<div style='width: 
170pt;word-wrap: break-word;white-space: normal'>A comma separated list of 
exception messages. If there is any exception thrown, whose message or 
stacktrace matches the specified message list, the engine would deregister 
itself.</div>|<div style='width: 30pt'>seq</div>|<div style='width: 
20pt'>1.2.0</div>
 kyuubi\.engine<br>\.deregister\.exception<br>\.ttl|<div style='width: 
65pt;word-wrap: break-word;white-space: normal'>PT30M</div>|<div style='width: 
170pt;word-wrap: break-word;white-space: normal'>Time to live(TTL) for 
exceptions pattern specified in kyuubi.engine.deregister.exception.classes and 
kyuubi.engine.deregister.exception.messages to deregister engines. Once the 
total error count hits the kyuubi.engine.deregister.job.max.failures within the 
TTL, an engine will deregister itself [...]
 kyuubi\.engine<br>\.deregister\.job\.max<br>\.failures|<div style='width: 
65pt;word-wrap: break-word;white-space: normal'>4</div>|<div style='width: 
170pt;word-wrap: break-word;white-space: normal'>Number of failures of job 
before deregistering the engine.</div>|<div style='width: 30pt'>int</div>|<div 
style='width: 20pt'>1.2.0</div>
-kyuubi\.engine\.event<br>\.json\.log\.path|<div style='width: 65pt;word-wrap: 
break-word;white-space: normal'>/tmp/kyuubi/events</div>|<div style='width: 
170pt;word-wrap: break-word;white-space: normal'>The location of all the engine 
events go for the builtin JSON logger.</div>|<div style='width: 
30pt'>string</div>|<div style='width: 20pt'>1.3.0</div>
+kyuubi\.engine\.event<br>\.json\.log\.path|<div style='width: 65pt;word-wrap: 
break-word;white-space: normal'>file:/tmp/kyuubi/events</div>|<div 
style='width: 170pt;word-wrap: break-word;white-space: normal'>The location of 
all the engine events go for the builtin JSON logger.<ul><li>Local Path: start 
with 'file:'</li><li>HDFS Path: start with 'hdfs:'</li></ul></div>|<div 
style='width: 30pt'>string</div>|<div style='width: 20pt'>1.3.0</div>
 kyuubi\.engine\.event<br>\.loggers|<div style='width: 65pt;word-wrap: 
break-word;white-space: normal'></div>|<div style='width: 170pt;word-wrap: 
break-word;white-space: normal'>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> <li>JSON: the events 
will be written to the location of kyuubi.engine.event.json.log.path</li> 
<li>JDBC: to be done</li> <li>CUSTOM: to be done. [...]
 kyuubi\.engine<br>\.initialize\.sql|<div style='width: 65pt;word-wrap: 
break-word;white-space: normal'>SHOW DATABASES</div>|<div style='width: 
170pt;word-wrap: break-word;white-space: normal'>SemiColon-separated list of 
SQL statements to be initialized in the newly created engine before queries. 
This configuration can not be used in JDBC url due to the limitation of 
Beeline/JDBC driver.</div>|<div style='width: 30pt'>seq</div>|<div 
style='width: 20pt'>1.2.0</div>
 kyuubi\.engine\.pool<br>\.size|<div style='width: 65pt;word-wrap: 
break-word;white-space: normal'>-1</div>|<div style='width: 170pt;word-wrap: 
break-word;white-space: normal'>The size of engine pool. Note that, if the size 
is less than 1, the engine pool will not be enabled; otherwise, the size of the 
engine pool will be min(this, kyuubi.engine.pool.size.threshold).</div>|<div 
style='width: 30pt'>int</div>|<div style='width: 20pt'>1.4.0</div>
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/EventLoggingServiceSuite.scala
index ae72b9d..b8926bb 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/EventLoggingServiceSuite.scala
@@ -17,9 +17,11 @@
 
 package org.apache.kyuubi.engine.spark.events
 
-import java.nio.charset.StandardCharsets
-import java.nio.file.{Files, Paths}
+import java.io.{BufferedReader, InputStreamReader}
+import java.nio.file.Paths
 
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path}
 import org.apache.hive.service.rpc.thrift.TExecuteStatementReq
 import org.scalatest.time.SpanSugar._
 
@@ -32,24 +34,27 @@ import org.apache.kyuubi.operation.{JDBCTestUtils, 
OperationHandle}
 
 class EventLoggingServiceSuite extends WithSparkSQLEngine with JDBCTestUtils {
 
-  private val logRoot = Utils.createTempDir()
+  private val logRoot = "file:" + Utils.createTempDir().toString
   private val currentDate = 
Utils.getDateFromTimestamp(System.currentTimeMillis())
 
   override def withKyuubiConf: Map[String, String] = Map(
     KyuubiConf.ENGINE_EVENT_LOGGERS.key -> s"$JSON,$SPARK",
-    KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH.key -> logRoot.toString,
+    KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH.key -> logRoot,
     "spark.eventLog.enabled" -> "true",
-    "spark.eventLog.dir" -> logRoot.toString
+    "spark.eventLog.dir" -> logRoot
   )
 
   override protected def jdbcUrl: String = getJdbcUrl
 
   test("round-trip for event logging service") {
     val engineEventPath = Paths.get(
-      logRoot.toString, "engine", s"day=$currentDate", 
KyuubiSparkUtil.engineId + ".json")
+      logRoot, "engine", s"day=$currentDate", KyuubiSparkUtil.engineId + 
".json")
     val sessionEventPath = Paths.get(
-      logRoot.toString, "session", s"day=$currentDate", 
KyuubiSparkUtil.engineId + ".json")
-    val engineEventReader = Files.newBufferedReader(engineEventPath, 
StandardCharsets.UTF_8)
+      logRoot, "session", s"day=$currentDate", KyuubiSparkUtil.engineId + 
".json")
+
+    val fileSystem: FileSystem = FileSystem.get(new Configuration())
+    val fs: FSDataInputStream = fileSystem.open(new 
Path(engineEventPath.toString))
+    val engineEventReader = new BufferedReader(new InputStreamReader(fs))
 
     val readEvent = JsonProtocol.jsonToEvent(engineEventReader.readLine(),
       classOf[KyuubiSparkEvent])
@@ -94,8 +99,8 @@ class EventLoggingServiceSuite extends WithSparkSQLEngine 
with JDBCTestUtils {
 
   test("statementEvent: generate, dump and query") {
     val statementEventPath = Paths.get(
-      logRoot.toString, "spark_statement", s"day=$currentDate", 
engine.engineId + ".json")
-    val sql = "select timestamp'2021-06-01'"
+      logRoot, "spark_statement", s"day=$currentDate", engine.engineId + 
".json")
+    val sql = "select timestamp'2021-09-01';"
     withSessionHandle { (client, handle) =>
 
       val table = statementEventPath.getParent
diff --git 
a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala 
b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index 05b8aa6..7c50821 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -830,10 +830,12 @@ object KyuubiConf {
 
   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.")
+      .doc("The location of all the engine events go for the builtin JSON 
logger.<ul>" +
+        "<li>Local Path: start with 'file:'</li>" +
+        "<li>HDFS Path: start with 'hdfs:'</li></ul>")
       .version("1.3.0")
       .stringConf
-      .createWithDefault("/tmp/kyuubi/events")
+      .createWithDefault("file:/tmp/kyuubi/events")
 
   val SERVER_EVENT_LOGGERS: ConfigEntry[Seq[String]] =
     buildConf("backend.server.event.loggers")
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 0f97b01..1890c00 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
@@ -19,7 +19,6 @@ package org.apache.kyuubi.events
 
 import java.io.{BufferedOutputStream, FileOutputStream, IOException, 
PrintWriter}
 import java.net.URI
-import java.nio.file.Paths
 
 import scala.collection.mutable.HashMap
 
@@ -83,7 +82,7 @@ class JsonEventLogger[T <: KyuubiEvent](logName: String,
   }
 
   override def initialize(conf: KyuubiConf): Unit = synchronized {
-    logRoot = Paths.get(conf.get(logPath)).toAbsolutePath.toUri
+    logRoot = URI.create(conf.get(logPath))
     fs = FileSystem.get(logRoot, hadoopConf)
     requireLogRootWritable()
     super.initialize(conf)
diff --git 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
index 4dc182f..55b1991 100644
--- 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
+++ 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/events/EventLoggingServiceSuite.scala
@@ -28,15 +28,15 @@ import org.apache.kyuubi.operation.OperationState._
 
 class EventLoggingServiceSuite extends WithKyuubiServer with JDBCTestUtils {
 
-  private val logRoot = Utils.createTempDir()
+  private val logRoot = "file:" + Utils.createTempDir().toString
   private val currentDate = 
Utils.getDateFromTimestamp(System.currentTimeMillis())
 
   override protected val conf: KyuubiConf = {
     KyuubiConf()
       .set(KyuubiConf.SERVER_EVENT_LOGGERS, Seq("JSON"))
-      .set(KyuubiConf.SERVER_EVENT_JSON_LOG_PATH, logRoot.toString)
+      .set(KyuubiConf.SERVER_EVENT_JSON_LOG_PATH, logRoot)
       .set(KyuubiConf.ENGINE_EVENT_LOGGERS, Seq("JSON"))
-      .set(KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH, logRoot.toString)
+      .set(KyuubiConf.ENGINE_EVENT_JSON_LOG_PATH, logRoot)
   }
 
   override protected def jdbcUrl: String = getJdbcUrl
@@ -44,9 +44,9 @@ class EventLoggingServiceSuite extends WithKyuubiServer with 
JDBCTestUtils {
   test("statementEvent: generate, dump and query") {
     val hostName = InetAddress.getLocalHost.getCanonicalHostName
     val serverStatementEventPath =
-      Paths.get(logRoot.toString, "kyuubi_statement", s"day=$currentDate", 
s"server-$hostName.json")
+      Paths.get(logRoot, "kyuubi_statement", s"day=$currentDate", 
s"server-$hostName.json")
     val engineStatementEventPath =
-      Paths.get(logRoot.toString, "spark_statement", s"day=$currentDate", 
"*.json")
+      Paths.get(logRoot, "spark_statement", s"day=$currentDate", "*.json")
     val sql = "select timestamp'2021-06-01'"
 
     withJdbcStatement() { statement =>
@@ -89,7 +89,7 @@ class EventLoggingServiceSuite extends WithKyuubiServer with 
JDBCTestUtils {
     }
 
     val eventPath =
-      Paths.get(logRoot.toString, "kyuubi_session", s"day=$currentDate")
+      Paths.get(logRoot, "kyuubi_session", s"day=$currentDate")
     withSessionConf()(Map.empty)(Map("spark.sql.shuffle.partitions" -> "2")) {
       withJdbcStatement() { statement =>
         val res = statement.executeQuery(
@@ -121,9 +121,9 @@ class EventLoggingServiceSuite extends WithKyuubiServer 
with JDBCTestUtils {
     }
 
     val serverSessionEventPath =
-      Paths.get(logRoot.toString, "kyuubi_session", s"day=$currentDate")
+      Paths.get(logRoot, "kyuubi_session", s"day=$currentDate")
     val engineSessionEventPath =
-      Paths.get(logRoot.toString, "session", s"day=$currentDate")
+      Paths.get(logRoot, "session", s"day=$currentDate")
     withSessionConf()(Map.empty)(Map.empty) {
       withJdbcStatement() { statement =>
         val res = statement.executeQuery(

Reply via email to