Skip to content

Commit 4148e5b

Browse files
wForgetulysses-you
authored andcommitted
[KYUUBI #1960] Skip creating sparksession and starting engine if the max initialization time exceeds
<!-- 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. --> close #1960 ### _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 Test steps: 1. Start another application to fill the queue. 2. Modify the `stop-application.sh` so it doesn't work. 3. Submit a Kyuubi request. 4. After waiting for timeout, stop the application in step 1. We can get the following results: 1. The kyuubi request exits after timeout. 2. After obtaining resources, the orphaned engine app fails to execute and exits immediately. ![image](https://user-images.githubusercontent.com/17894939/155467408-f9c5d9d3-ccb9-47a6-b697-1d64f42e427e.png) ![image](https://user-images.githubusercontent.com/17894939/155467301-7ee01616-10dd-4c25-9375-02c0ae88091e.png) - [ ] [Run test](https://kyuubi.apache.org/docs/latest/develop_tools/testing.html#running-tests) locally before make a pull request Closes #1966 from wForget/KYUUBI-1960. Closes #1960 0f977c8 [Wang Zhen] spotless 9bb2178 [Wang Zhen] Merge branch 'master' of https://github.com/apache/incubator-kyuubi into KYUUBI-1960 3effa4d [wForget] fix f9f13c6 [wForget] fix test abdc1f3 [wForget] fix 48b4bac [wForget] [KYUUBI-1960] Skip creating sparksession and starting engine if the max initialization time exceeds Lead-authored-by: wForget <643348094@qq.com> Co-authored-by: Wang Zhen <wangzhen07@qiyi.com> Signed-off-by: ulysses-you <ulyssesyou@apache.org>
1 parent 6d94fee commit 4148e5b

File tree

4 files changed

+50
-25
lines changed

4 files changed

+50
-25
lines changed

externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/SparkSQLEngine.scala

Lines changed: 45 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -31,6 +31,7 @@ import org.apache.kyuubi.{KyuubiException, Logging, Utils}
3131
import org.apache.kyuubi.Utils._
3232
import org.apache.kyuubi.config.KyuubiConf
3333
import org.apache.kyuubi.config.KyuubiConf._
34+
import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_SUBMIT_TIME_KEY
3435
import org.apache.kyuubi.engine.spark.SparkSQLEngine.{countDownLatch, currentEngine}
3536
import org.apache.kyuubi.engine.spark.events.{EngineEvent, EngineEventsStore, EventLoggingService}
3637
import org.apache.kyuubi.events.EventLogging
@@ -70,6 +71,8 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
7071

7172
object SparkSQLEngine extends Logging {
7273

74+
val sparkConf: SparkConf = new SparkConf()
75+
7376
val kyuubiConf: KyuubiConf = KyuubiConf()
7477

7578
var currentEngine: Option[SparkSQLEngine] = None
@@ -78,8 +81,7 @@ object SparkSQLEngine extends Logging {
7881

7982
private val countDownLatch = new CountDownLatch(1)
8083

81-
def createSpark(): SparkSession = {
82-
val sparkConf = new SparkConf()
84+
def setupConf(): Unit = {
8385
val rootDir = sparkConf.getOption("spark.repl.classdir").getOrElse(getLocalDir(sparkConf))
8486
val outputDir = Utils.createTempDir(root = rootDir, namePrefix = "repl")
8587
sparkConf.setIfMissing("spark.sql.execution.topKSortFallbackThreshold", "10000")
@@ -112,11 +114,16 @@ object SparkSQLEngine extends Logging {
112114
debug(s"KyuubiConf: $k = $v")
113115
}
114116
}
117+
}
115118

119+
def createSpark(): SparkSession = {
116120
val session = SparkSession.builder.config(sparkConf).getOrCreate
117121
(kyuubiConf.get(ENGINE_INITIALIZE_SQL) ++ kyuubiConf.get(ENGINE_SESSION_INITIALIZE_SQL))
118122
.foreach { sqlStr =>
119-
session.sparkContext.setJobGroup(appName, sqlStr, interruptOnCancel = true)
123+
session.sparkContext.setJobGroup(
124+
"engine_initializing_queries",
125+
sqlStr,
126+
interruptOnCancel = true)
120127
debug(s"Execute session initializing sql: $sqlStr")
121128
session.sql(sqlStr).isEmpty
122129
}
@@ -168,29 +175,43 @@ object SparkSQLEngine extends Logging {
168175

169176
def main(args: Array[String]): Unit = {
170177
SignalRegister.registerLogger(logger)
171-
var spark: SparkSession = null
172-
try {
173-
spark = createSpark()
178+
setupConf()
179+
val startedTime = System.currentTimeMillis()
180+
val submitTime = kyuubiConf.getOption(KYUUBI_ENGINE_SUBMIT_TIME_KEY) match {
181+
case Some(t) => t.toLong
182+
case _ => startedTime
183+
}
184+
val initTimeout = kyuubiConf.get(ENGINE_INIT_TIMEOUT)
185+
val totalInitTime = startedTime - submitTime
186+
if (totalInitTime > initTimeout) {
187+
throw new KyuubiException(s"The total engine initialization time ($totalInitTime ms)" +
188+
s" exceeds `kyuubi.session.engine.initialize.timeout` ($initTimeout ms)," +
189+
s" and submitted at $submitTime.")
190+
} else {
191+
var spark: SparkSession = null
174192
try {
175-
startEngine(spark)
176-
// blocking main thread
177-
countDownLatch.await()
193+
spark = createSpark()
194+
try {
195+
startEngine(spark)
196+
// blocking main thread
197+
countDownLatch.await()
198+
} catch {
199+
case e: KyuubiException => currentEngine match {
200+
case Some(engine) =>
201+
engine.stop()
202+
val event = EngineEvent(engine).copy(diagnostic = e.getMessage)
203+
EventLogging.onEvent(event)
204+
error(event, e)
205+
case _ => error("Current SparkSQLEngine is not created.")
206+
}
207+
208+
}
178209
} catch {
179-
case e: KyuubiException => currentEngine match {
180-
case Some(engine) =>
181-
engine.stop()
182-
val event = EngineEvent(engine).copy(diagnostic = e.getMessage)
183-
EventLogging.onEvent(event)
184-
error(event, e)
185-
case _ => error("Current SparkSQLEngine is not created.")
186-
}
187-
188-
}
189-
} catch {
190-
case t: Throwable => error(s"Failed to instantiate SparkSession: ${t.getMessage}", t)
191-
} finally {
192-
if (spark != null) {
193-
spark.stop()
210+
case t: Throwable => error(s"Failed to instantiate SparkSession: ${t.getMessage}", t)
211+
} finally {
212+
if (spark != null) {
213+
spark.stop()
214+
}
194215
}
195216
}
196217
}

externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/WithSparkSQLEngine.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -66,6 +66,7 @@ trait WithSparkSQLEngine extends KyuubiFunSuite {
6666

6767
SparkSession.clearActiveSession()
6868
SparkSession.clearDefaultSession()
69+
SparkSQLEngine.setupConf()
6970
spark = SparkSQLEngine.createSpark()
7071
SparkSQLEngine.startEngine(spark)
7172
engine = SparkSQLEngine.currentEngine.get

kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiReservedKeys.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -20,4 +20,5 @@ package org.apache.kyuubi.config
2020
object KyuubiReservedKeys {
2121
final val KYUUBI_SESSION_USER_KEY = "kyuubi.session.user"
2222
final val KYUUBI_STATEMENT_ID_KEY = "kyuubi.statement.id"
23+
final val KYUUBI_ENGINE_SUBMIT_TIME_KEY = "kyuubi.engine.submit.time"
2324
}

kyuubi-server/src/main/scala/org/apache/kyuubi/engine/EngineRef.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,7 @@ import org.apache.hadoop.security.UserGroupInformation
3232
import org.apache.kyuubi.{KYUUBI_VERSION, KyuubiSQLException, Logging, Utils}
3333
import org.apache.kyuubi.config.KyuubiConf
3434
import org.apache.kyuubi.config.KyuubiConf._
35+
import org.apache.kyuubi.config.KyuubiReservedKeys.KYUUBI_ENGINE_SUBMIT_TIME_KEY
3536
import org.apache.kyuubi.engine.EngineType.{EngineType, FLINK_SQL, SPARK_SQL, TRINO}
3637
import org.apache.kyuubi.engine.ShareLevel.{CONNECTION, GROUP, SERVER, ShareLevel}
3738
import org.apache.kyuubi.engine.flink.FlinkProcessBuilder
@@ -178,6 +179,8 @@ private[kyuubi] class EngineRef(
178179

179180
conf.set(HA_ZK_NAMESPACE, engineSpace)
180181
conf.set(HA_ZK_ENGINE_REF_ID, engineRefId)
182+
val started = System.currentTimeMillis()
183+
conf.set(KYUUBI_ENGINE_SUBMIT_TIME_KEY, String.valueOf(started))
181184
val builder = engineType match {
182185
case SPARK_SQL =>
183186
conf.setIfMissing(SparkProcessBuilder.APP_KEY, defaultEngineName)
@@ -203,7 +206,6 @@ private[kyuubi] class EngineRef(
203206
try {
204207
info(s"Launching engine:\n$builder")
205208
val process = builder.start
206-
val started = System.currentTimeMillis()
207209
var exitValue: Option[Int] = None
208210
while (engineRef.isEmpty) {
209211
if (exitValue.isEmpty && process.waitFor(1, TimeUnit.SECONDS)) {

0 commit comments

Comments
 (0)