Skip to content

Commit c9c882f

Browse files
refactor: Update shutdown watchdog timeout handling in SparkSQLEngine
- Changed the shutdown watchdog timeout configuration from minutes to milliseconds for more precise control. - Updated related logging messages to reflect the new timeout unit. - Adjusted the default value in KyuubiConf to align with the new configuration format.
1 parent c2868d2 commit c9c882f

3 files changed

Lines changed: 7 additions & 12 deletions

File tree

docs/configuration/settings.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -494,7 +494,7 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
494494
| kyuubi.session.engine.open.onFailure | RETRY | The behavior when opening engine failed: <ul> <li>RETRY: retry to open engine for kyuubi.session.engine.open.max.attempts times.</li> <li>DEREGISTER_IMMEDIATELY: deregister the engine immediately.</li> <li>DEREGISTER_AFTER_RETRY: deregister the engine after retry to open engine for kyuubi.session.engine.open.max.attempts times.</li></ul> | string | 1.8.1 |
495495
| kyuubi.session.engine.open.retry.wait | PT10S | How long to wait before retrying to open the engine after failure. | duration | 1.7.0 |
496496
| kyuubi.session.engine.share.level | USER | (deprecated) - Using kyuubi.engine.share.level instead | string | 1.0.0 |
497-
| kyuubi.session.engine.shutdown.watchdog.timeout | PT0.001S | The maximum time to wait for the engine to shutdown gracefully before forcing termination. When an engine shutdown is initiated, this watchdog timer starts counting down. If the engine doesn't complete shutdown within this timeout period, it will be forcefully terminated to prevent hanging. Set to 0 or a negative value to disable the forced shutdown mechanism. | duration | 1.11.0 |
497+
| kyuubi.session.engine.shutdown.watchdog.timeout | PT1M | The maximum time to wait for the engine to shutdown gracefully before forcing termination. When an engine shutdown is initiated, this watchdog timer starts counting down. If the engine doesn't complete shutdown within this timeout period, it will be forcefully terminated to prevent hanging. Set to 0 or a negative value to disable the forced shutdown mechanism. | duration | 1.11.0 |
498498
| kyuubi.session.engine.spark.initialize.sql || The initialize sql for Spark session. It fallback to `kyuubi.engine.session.initialize.sql` | seq | 1.8.1 |
499499
| kyuubi.session.engine.spark.main.resource | &lt;undefined&gt; | The package used to create Spark SQL engine remote application. If it is undefined, Kyuubi will use the default | string | 1.0.0 |
500500
| kyuubi.session.engine.spark.max.initial.wait | PT1M | 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. 0 or negative means not to self-terminate. | duration | 1.8.0 |

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

Lines changed: 5 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -228,9 +228,8 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
228228
return
229229
}
230230

231-
val shutdownWatchdogTimeoutMinutes = conf.get(ENGINE_SHUTDOWN_WATCHDOG_TIMEOUT)
232-
233-
if (shutdownWatchdogTimeoutMinutes <= 0) {
231+
val shutdownWatchdogTimeout = conf.get(ENGINE_SHUTDOWN_WATCHDOG_TIMEOUT)
232+
if (shutdownWatchdogTimeout <= 0) {
234233
info("Shutdown Watchdog is disabled (timeout <= 0).")
235234
return
236235
}
@@ -243,21 +242,17 @@ case class SparkSQLEngine(spark: SparkSession) extends Serverable("SparkSQLEngin
243242
}
244243
}
245244

246-
if (shutdownWatchdogTimeoutMinutes >= 60) {
247-
warn(s"Shutdown Watchdog timeout is very long (${shutdownWatchdogTimeoutMinutes} minutes).")
248-
}
249-
250245
info(s"Shutdown Watchdog activated. Engine will be forcefully terminated if graceful " +
251-
s"shutdown exceeds ${shutdownWatchdogTimeoutMinutes} minute(s).")
246+
s"shutdown exceeds ${shutdownWatchdogTimeout} ms.")
252247

253248
val watchdogThread = new Thread("shutdown-watchdog") {
254249
override def run(): Unit = {
255250
debug("Shutdown Watchdog thread started, monitoring graceful shutdown process")
256251
try {
257-
TimeUnit.MINUTES.sleep(shutdownWatchdogTimeoutMinutes)
252+
TimeUnit.MILLISECONDS.sleep(shutdownWatchdogTimeout)
258253

259254
error(s"EMERGENCY SHUTDOWN TRIGGERED")
260-
error(s"Graceful shutdown exceeded ${shutdownWatchdogTimeoutMinutes} minute(s) timeout")
255+
error(s"Graceful shutdown exceeded ${shutdownWatchdogTimeout} ms timeout")
261256
error(s"Non-daemon threads are preventing JVM exit")
262257
error(s"Initiating forced termination...")
263258

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1793,7 +1793,7 @@ object KyuubiConf {
17931793
"Set to 0 or a negative value to disable the forced shutdown mechanism.")
17941794
.version("1.11.0")
17951795
.timeConf
1796-
.createWithDefault(Duration.ofMinutes(1L).toMinutes)
1796+
.createWithDefault(Duration.ofMinutes(1L).toMillis)
17971797

17981798
val SESSION_CONF_IGNORE_LIST: ConfigEntry[Set[String]] =
17991799
buildConf("kyuubi.session.conf.ignore.list")

0 commit comments

Comments
 (0)