You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@kyuubi.apache.org by "risyomei (via GitHub)" <gi...@apache.org> on 2023/06/28 02:02:30 UTC

[GitHub] [kyuubi] risyomei opened a new pull request, #5002: Fail the engine fast when no incoming connection in CONNECTION mode

risyomei opened a new pull request, #5002:
URL: https://github.com/apache/kyuubi/pull/5002

   ### _Why are the changes needed?_
   Please refer to #4997
   
   ### _How was this patch tested?_
   - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible
   
   - [x] Add screenshots for manual tests if appropriate
   1. connect to KyuubiServer with beeline
   2. Confirm the Application is ACCEPTed in ResourceManager, Restart KyuubiServer
   3. Confirmed that Engine was terminated shortly
   ```
   23/06/28 10:44:59 INFO storage.BlockManagerMaster: Removed 1 successfully in removeExecutor
   23/06/28 10:45:00 INFO spark.SparkSQLEngine: Current open session is 0
   23/06/28 10:45:00 ERROR spark.SparkSQLEngine: Spark engine has been terminated because no incoming connection for more than 60000 ms, deregistering from engine discovery space.
   23/06/28 10:45:00 WARN zookeeper.ZookeeperDiscoveryClient: This Kyuubi instance lniuhpi1616.nhnjp.ism:46588 is now de-registered from ZooKeeper. The server will be shut down after the last client session completes.
   23/06/28 10:45:00 INFO spark.SparkSQLEngine: Service: [SparkTBinaryFrontend] is stopping.
   ``` 
   
   - [x] [Run test](https://kyuubi.readthedocs.io/en/master/contributing/code/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: notifications-unsubscribe@kyuubi.apache.org

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244662390


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -57,6 +58,7 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
 
   @volatile private var lifetimeTerminatingChecker: Option[ScheduledExecutorService] = None
   @volatile private var stopEngineExec: Option[ThreadPoolExecutor] = None
+  @volatile private var failfastChecker: Option[ScheduledExecutorService] = None

Review Comment:
   I will fix it.



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1248813471


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   The reason I disabled this feature because it seems to affect the CI/CD test.
   As I described in https://github.com/apache/kyuubi/pull/5002#issuecomment-1611305696, almost every test are failing. But they runs fine locally.
   
   Do you have any idea how I can improve it?



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

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

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


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


[GitHub] [kyuubi] risyomei commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1637662507

   Thank you for your review and comment.


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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1248813257


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")

Review Comment:
   Sure, I will fix it.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1261903864


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,52 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]

Review Comment:
   ```
     def runInNewThread(
         threadName: String,
         isDaemon: Boolean = true)(body: => Unit): Unit = {
       
       val thread = new Thread(threadName) {
         override def run(): Unit = {
           body
         }
       }
       thread.setDaemon(isDaemon)
       thread.setUncaughtExceptionHandler(NamedThreadFactory.kyuubiUncaughtExceptionHandler)
       thread.start()
     }
   ```



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1637114438

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

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


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


[GitHub] [kyuubi] cxzl25 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "cxzl25 (via GitHub)" <gi...@apache.org>.
cxzl25 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1250086894


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,53 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]
+
+    val thread = new Thread(threadName) {
+      override def run(): Unit = {
+        try {
+          result = body
+        } catch {
+          case NonFatal(e) =>
+            exception = Some(e)
+        }
+      }
+    }
+    thread.setDaemon(isDaemon)
+    thread.start()
+    thread.join()

Review Comment:
   The join method here will cause the engine to get stuck.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1247616941


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   why disable if in default? 60s should be a good a default value as we discussed before



##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   why disable if in default? 60s should be a good default value as we discussed before



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1247615217


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -114,6 +127,29 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
     stopEngineExec.get.execute(stopTask)
   }
 
+  private[kyuubi] def startFastFailChecker(): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val timeout = conf.get(ENGINE_SPARK_MAX_INITIAL_WAIT)
+    val failFastCheck: Runnable = () => {
+      if (!shutdown.get) {
+        while (backendService.sessionManager.getOpenSessionCount <= 0 &&
+          System.currentTimeMillis() - getStartTime < timeout) {
+          info(s"Waiting for the initial connection")
+          Thread.sleep(interval)
+        }
+        if (backendService.sessionManager.getOpenSessionCount <= 0) {
+          error(s"Spark engine has been terminated because no incoming connection" +
+            s" for more than $timeout ms, de-registering from engine discovery space.")
+          assert(currentEngine.isDefined)
+          currentEngine.get.stop()
+        }
+      }
+    }
+    failfastChecker =
+      Some(ThreadUtils.newDaemonFixedThreadPool(1, "spark-engine-failfast-checker"))

Review Comment:
   We don't need a thread pool, let's port `ThreadUtils#runInNewThread` from Spark



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1264457607


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,52 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]

Review Comment:
   Thank you very much for your detailed explanation. I have fixed this method based on your comment.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244588342


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val deregistered = new AtomicBoolean(false)
+    var timeout = Duration(60, TimeUnit.SECONDS).toMillis

Review Comment:
   60s may be fine, but I think it should be configurable.



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

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

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


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


[GitHub] [kyuubi] risyomei commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1611305696

   I noticed that many tests are failed on `SparkOperationSuite`, but the test runs fine locally.
   https://gist.github.com/risyomei/d602a86d14f4d72170701dadb528559a
   
   I am still debugging, any advice is welcomed.


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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1261902582


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,52 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]
+
+    val thread = new Thread(threadName) {
+      override def run(): Unit = {
+        try {
+          result = body
+        } catch {
+          case NonFatal(e) =>
+            exception = Some(e)
+        }
+      }
+    }
+    thread.setDaemon(isDaemon)

Review Comment:
   ```
   thread.setUncaughtExceptionHandler(NamedThreadFactory.kyuubiUncaughtExceptionHandler)
   ```



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1248813471


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   ~The reason I disabled this feature because it seems to affect the CI/CD test.
   As I described in https://github.com/apache/kyuubi/pull/5002#issuecomment-1611305696, almost every test are failing. But they runs fine locally.~
   
   ~Do you have any idea how I can improve it?~
   
   I have slightly modified the existing tests



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244575706


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)

Review Comment:
   Still using the `ENGINE_CHECK_INTERVAL`, but I am open to add a new configuration key



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val deregistered = new AtomicBoolean(false)
+    var timeout = Duration(60, TimeUnit.SECONDS).toMillis

Review Comment:
   Using a fixed duration 60 seconds, but I am open to add a new configuration key.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1247633347


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")

Review Comment:
   we should mention 0 means disable too.



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

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

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


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


[GitHub] [kyuubi] pan3793 closed pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 closed pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode
URL: https://github.com/apache/kyuubi/pull/5002


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

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

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


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


[GitHub] [kyuubi] pan3793 commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1633445139

   Except for the return value of `runInNewThread`, other things LGTM.


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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1261901002


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,52 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]

Review Comment:
   in this case, this is kind of "run something in a new thread and forget it". we don't care about the result.
   
   since we don't want to block the current thread, the result T or exception still be the initiate value after the function return.



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1248813471


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   Sure, let me fix it.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1247629794


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")

Review Comment:
   nit: space



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244576837


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -80,6 +82,13 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
       assert(currentEngine.isDefined)
       currentEngine.get.stop()
     })
+
+    if (conf.get(ENGINE_SHARE_LEVEL) == ShareLevel.CONNECTION.toString) {

Review Comment:
   Initiate this check only when the share level is CONNECTION



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244585916


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -57,6 +58,7 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
 
   @volatile private var lifetimeTerminatingChecker: Option[ScheduledExecutorService] = None
   @volatile private var stopEngineExec: Option[ThreadPoolExecutor] = None
+  @volatile private var failfastChecker: Option[ScheduledExecutorService] = None

Review Comment:
   do we need to schedule the task? it can be exited after the session is created.



##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)

Review Comment:
   I think it's fine



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1248813072


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -114,6 +127,29 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
     stopEngineExec.get.execute(stopTask)
   }
 
+  private[kyuubi] def startFastFailChecker(): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val timeout = conf.get(ENGINE_SPARK_MAX_INITIAL_WAIT)
+    val failFastCheck: Runnable = () => {
+      if (!shutdown.get) {
+        while (backendService.sessionManager.getOpenSessionCount <= 0 &&
+          System.currentTimeMillis() - getStartTime < timeout) {
+          info(s"Waiting for the initial connection")
+          Thread.sleep(interval)
+        }
+        if (backendService.sessionManager.getOpenSessionCount <= 0) {
+          error(s"Spark engine has been terminated because no incoming connection" +
+            s" for more than $timeout ms, de-registering from engine discovery space.")
+          assert(currentEngine.isDefined)
+          currentEngine.get.stop()
+        }
+      }
+    }
+    failfastChecker =
+      Some(ThreadUtils.newDaemonFixedThreadPool(1, "spark-engine-failfast-checker"))

Review Comment:
   Got it, let me fix it.



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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1250297612


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/util/ThreadUtils.scala:
##########
@@ -95,4 +96,53 @@ object ThreadUtils extends Logging {
       }
     }
   }
+
+  def runInNewThread[T](
+      threadName: String,
+      isDaemon: Boolean = true)(body: => T): T = {
+    @volatile var exception: Option[Throwable] = None
+    @volatile var result: T = null.asInstanceOf[T]
+
+    val thread = new Thread(threadName) {
+      override def run(): Unit = {
+        try {
+          result = body
+        } catch {
+          case NonFatal(e) =>
+            exception = Some(e)
+        }
+      }
+    }
+    thread.setDaemon(isDaemon)
+    thread.start()
+    thread.join()

Review Comment:
   Yes, that's ture. Thank you for pointing out.
   Shall I change the name of `runInNewThread` as well? (So that others may notice the difference with the runInNewThread in Spark.)



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

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

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


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


[GitHub] [kyuubi] risyomei commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1631895548

   @cxzl25 @pan3793 
   
   I am not meant to push but may I ask you check this at your earliest convenience?


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

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

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


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


[GitHub] [kyuubi] pan3793 commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1631897069

   @risyomei Sorry, was busy with internal stuff these days, will take a look soon.


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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244588342


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val deregistered = new AtomicBoolean(false)
+    var timeout = Duration(60, TimeUnit.SECONDS).toMillis

Review Comment:
   60s may be fine, but I think it should be configurable (we can also use 0 to disable this feature).



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

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

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


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


[GitHub] [kyuubi] codecov-commenter commented on pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#issuecomment-1610628459

   ## [Codecov](https://app.codecov.io/gh/apache/kyuubi/pull/5002?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   > Merging [#5002](https://app.codecov.io/gh/apache/kyuubi/pull/5002?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (639bd52) into [master](https://app.codecov.io/gh/apache/kyuubi/commit/98558025056ff84b9d4f1cb770d64f77000d1142?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) (9855802) will **not change** coverage.
   > The diff coverage is `0.00%`.
   
   ```diff
   @@          Coverage Diff           @@
   ##           master   #5002   +/-   ##
   ======================================
     Coverage    0.00%   0.00%           
   ======================================
     Files         563     563           
     Lines       31167   31196   +29     
     Branches     4072    4076    +4     
   ======================================
   - Misses      31167   31196   +29     
   ```
   
   
   | [Impacted Files](https://app.codecov.io/gh/apache/kyuubi/pull/5002?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [...rg/apache/kyuubi/engine/spark/SparkSQLEngine.scala](https://app.codecov.io/gh/apache/kyuubi/pull/5002?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-ZXh0ZXJuYWxzL2t5dXViaS1zcGFyay1zcWwtZW5naW5lL3NyYy9tYWluL3NjYWxhL29yZy9hcGFjaGUva3l1dWJpL2VuZ2luZS9zcGFyay9TcGFya1NRTEVuZ2luZS5zY2FsYQ==) | `0.00% <0.00%> (ø)` | |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache)
   


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

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

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


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


[GitHub] [kyuubi] risyomei commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "risyomei (via GitHub)" <gi...@apache.org>.
risyomei commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1244604722


##########
externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala:
##########
@@ -147,6 +162,34 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
         TimeUnit.MILLISECONDS)
     }
   }
+
+  private[kyuubi] def startFailFastChecker(stop: () => Unit): Unit = {
+    val interval = conf.get(ENGINE_CHECK_INTERVAL)
+    val deregistered = new AtomicBoolean(false)
+    var timeout = Duration(60, TimeUnit.SECONDS).toMillis

Review Comment:
   Got it, I will add it in the next commit.



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

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

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


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


[GitHub] [kyuubi] pan3793 commented on a diff in pull request #5002: Fail the engine fast when no incoming connection in CONNECTION mode

Posted by "pan3793 (via GitHub)" <gi...@apache.org>.
pan3793 commented on code in PR #5002:
URL: https://github.com/apache/kyuubi/pull/5002#discussion_r1247616941


##########
kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala:
##########
@@ -1282,6 +1282,15 @@ object KyuubiConf {
       .timeConf
       .createWithDefault(0)
 
+  val ENGINE_SPARK_MAX_INITIAL_WAIT: ConfigEntry[Long] =
+    buildConf("kyuubi.session.engine.spark.max.initial.wait")
+      .doc("Max wait time for the initial connection to Spark engine. The engine will" +
+        " self-terminate no new incoming connection is established within this time." +
+        "This setting only applies at the CONNECTION share level.")
+      .version("1.8.0")
+      .timeConf
+      .createWithDefault(Duration.ofSeconds(0).toMillis)

Review Comment:
   why disable it in default? 60s should be a good default value as we discussed before



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

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

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


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