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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7c5f58384 [KYUUBI #5331] Spark engine should throw an exception when 
it fails to start
7c5f58384 is described below

commit 7c5f5838420dcb1931031391aeee82ba492685c6
Author: liupeiyue <[email protected]>
AuthorDate: Mon Nov 13 19:47:05 2023 +0800

    [KYUUBI #5331] Spark engine should throw an exception when it fails to start
    
    ### _Why are the changes needed?_
    
    Close #5331
    
    ### _How was this patch tested?_
    - [x] 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/master/contributing/code/testing.html#running-tests)
 locally before make a pull request
    
    ### _Was this patch authored or co-authored using generative AI tooling?_
    
    No.
    
    Closes #5332 from ASiegeLion/master.
    
    Closes #5331
    
    21342f546 [sychen] wrap InterruptedException
    1f2542c82 [sychen] fix UT
    e433b544f [liupeiyue] [KYUUBI #5331]Spark Engine should throw an exception 
to let K8s know when the engine fails to start
    
    Lead-authored-by: liupeiyue <[email protected]>
    Co-authored-by: sychen <[email protected]>
    Signed-off-by: Cheng Pan <[email protected]>
---
 .../org/apache/kyuubi/engine/spark/SparkSQLEngine.scala  | 16 +++++++++++-----
 .../kyuubi/engine/spark/IndividualSparkSuite.scala       |  6 +++++-
 .../scala/org/apache/kyuubi/WithKyuubiServerOnYarn.scala |  2 +-
 3 files changed, 17 insertions(+), 7 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 ba84e1b1b..6e323cfe7 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
@@ -361,7 +361,8 @@ object SparkSQLEngine extends Logging {
           // blocking main thread
           countDownLatch.await()
         } catch {
-          case e: KyuubiException => currentEngine match {
+          case e: KyuubiException =>
+            currentEngine match {
               case Some(engine) =>
                 engine.stop()
                 val event = EngineEvent(engine)
@@ -370,16 +371,21 @@ object SparkSQLEngine extends Logging {
                 error(event, e)
               case _ => error("Current SparkSQLEngine is not created.")
             }
+            throw e
 
         }
       } catch {
         case i: InterruptedException if !sparkSessionCreated.get =>
-          error(
+          val msg =
             s"The Engine main thread was interrupted, possibly due to 
`createSpark` timeout." +
               s" The `${ENGINE_INIT_TIMEOUT.key}` is ($initTimeout ms) " +
-              s" and submitted at $submitTime.",
-            i)
-        case t: Throwable => error(s"Failed to instantiate SparkSession: 
${t.getMessage}", t)
+              s" and submitted at $submitTime."
+          error(msg, i)
+          throw new InterruptedException(msg)
+        case e: KyuubiException => throw e
+        case t: Throwable =>
+          error(s"Failed to instantiate SparkSession: ${t.getMessage}", t)
+          throw t
       } finally {
         if (spark != null) {
           spark.stop()
diff --git 
a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/IndividualSparkSuite.scala
 
b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/IndividualSparkSuite.scala
index 8fca1d0ca..8376705ef 100644
--- 
a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/IndividualSparkSuite.scala
+++ 
b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/IndividualSparkSuite.scala
@@ -110,7 +110,11 @@ class SparkEngineSuites extends KyuubiFunSuite {
       SparkSQLEngine.currentEngine = None
       val logAppender = new LogAppender("test createSpark timeout")
       withLogAppender(logAppender) {
-        SparkSQLEngine.main(Array.empty)
+        try {
+          SparkSQLEngine.main(Array.empty)
+        } catch {
+          case e: Exception => error("", e)
+        }
       }
       assert(SparkSQLEngine.currentEngine.isEmpty)
       val errorMsg = s"The Engine main thread was interrupted, possibly due to 
`createSpark`" +
diff --git 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/WithKyuubiServerOnYarn.scala 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/WithKyuubiServerOnYarn.scala
index 012f4df16..5a674d98f 100644
--- 
a/kyuubi-server/src/test/scala/org/apache/kyuubi/WithKyuubiServerOnYarn.scala
+++ 
b/kyuubi-server/src/test/scala/org/apache/kyuubi/WithKyuubiServerOnYarn.scala
@@ -206,7 +206,7 @@ class KyuubiOperationYarnClusterSuite extends 
WithKyuubiServerOnYarn with HiveJD
       }
       val elapsedTime = System.currentTimeMillis() - startTime
       assert(elapsedTime < 60 * 1000)
-      assert(exception.getMessage contains "The engine application has been 
terminated.")
+      assert(exception.getMessage contains "Could not open client transport 
with JDBC Uri")
     }
   }
 }

Reply via email to