diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala index a78166120..45924aef6 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/Utils.scala @@ -26,6 +26,7 @@ import scala.collection.JavaConverters._ import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.util.ShutdownHookManager import org.apache.kyuubi.config.internal.Tests.IS_TESTING @@ -165,4 +166,13 @@ private[kyuubi] object Utils extends Logging { def isTesting: Boolean = { System.getProperty(IS_TESTING.key) != null } + + /** + * Add some operations that you want into ShutdownHook + * @param hook + * @param priority: 0~100 + */ + def addShutdownHook(hook: Runnable, priority: Int): Unit = { + ShutdownHookManager.get().addShutdownHook(hook, priority) + } } diff --git a/kyuubi-main/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala b/kyuubi-main/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala index eb9bc0503..380df1674 100644 --- a/kyuubi-main/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala +++ b/kyuubi-main/src/main/scala/org/apache/kyuubi/server/KyuubiServer.scala @@ -44,7 +44,9 @@ object KyuubiServer extends Logging { val server = new KyuubiServer() server.initialize(conf) server.start() - sys.addShutdownHook(server.stop()) + Utils.addShutdownHook(new Runnable { + override def run(): Unit = server.stop() + }, 100) server } diff --git a/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/EmbeddedZookeeper.scala b/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/EmbeddedZookeeper.scala index 6c7d2af4f..475afbd6d 100644 --- a/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/EmbeddedZookeeper.scala +++ b/kyuubi-zookeeper/src/main/scala/org/apache/kyuubi/zookeeper/EmbeddedZookeeper.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.curator.test.{InstanceSpec, TestingServer} +import org.apache.kyuubi.Utils import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.service.{AbstractService, ServiceState} import org.apache.kyuubi.zookeeper.ZookeeperConf._ @@ -71,7 +72,9 @@ class EmbeddedZookeeper extends AbstractService("EmbeddedZookeeper") { override def start(): Unit = synchronized { server.start() info(s"$getName is started at $getConnectString") - sys.addShutdownHook(server.close()) + Utils.addShutdownHook(new Runnable { + override def run(): Unit = server.close() + }, 50) super.start() }