From 7deb2f7fd7e0f5db3f9ae0f5741371ec4bab3973 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Tue, 2 Feb 2021 20:46:06 +0800 Subject: [PATCH] [KYUUBI #336] Fix a race in Kyuubi start and stop ![yaooqinn](https://badgen.net/badge/Hello/yaooqinn/green) [![Closes #336](https://badgen.net/badge/Preview/Closes%20%23336/blue)](https://github.com/yaooqinn/kyuubi/pull/336) ![29](https://badgen.net/badge/%2B/29/red) ![36](https://badgen.net/badge/-/36/green) ![10](https://badgen.net/badge/commits/10/yellow) ![Target Issue](https://badgen.net/badge/Missing/Target%20Issue/ff0000) [❨?❩](https://pullrequestbadge.com/?utm_medium=github&utm_source=yaooqinn&utm_campaign=badge_info) ### _Why are the changes needed?_ there is something wrong with Travis these days due to some newly added test cases ### _How was this patch tested?_ - [ ] Add some test cases that check the changes thoroughly including negative and positive cases if possible - [ ] Add screenshots for manual tests if appropriate - [x] [Run test](https://kyuubi.readthedocs.io/en/latest/tools/testing.html#running-tests) locally before make a pull request Closes #336 from yaooqinn/travisagain. 0285f5a [Kent Yao] fix a race 3624889 [Kent Yao] fix a race 895b4d5 [Kent Yao] fix a race e253b9c [Kent Yao] fix a race 75d699a [Kent Yao] fix a race 7bde6b9 [Kent Yao] nit 6959e42 [Kent Yao] nit 062a151 [Kent Yao] nit ca8af10 [Kent Yao] kyuubi-common 6e4b8b6 [jhx1008] Try to restore travis Lead-authored-by: Kent Yao Co-authored-by: jhx1008 Signed-off-by: Kent Yao --- .gitignore | 4 +--- .travis.yml | 16 ++++++------- .../spark/SparkSQLEngineListenerSuite.scala | 18 +-------------- .../engine/spark/WithSparkSQLEngine.scala | 4 ++++ .../kyuubi/service/FrontendService.scala | 23 ++++++++++++------- 5 files changed, 29 insertions(+), 36 deletions(-) diff --git a/.gitignore b/.gitignore index f6592756f..bc2891418 100644 --- a/.gitignore +++ b/.gitignore @@ -47,6 +47,4 @@ embedded_zookeeper/ /work/ /docs/_build/ /kyuubi-common/metrics/ -kyuubi-common/operation_logs/ -/kyuubi-common/operation_logs/ -/kyuubi-main/operation_logs/ +**/operation_logs/ diff --git a/.travis.yml b/.travis.yml index 65f5a0240..bf391538b 100644 --- a/.travis.yml +++ b/.travis.yml @@ -44,15 +44,15 @@ deploy: # tags: true install: - - build/mvn --version - - build/mvn clean + - mvn --version + - mvn clean before_script: - - build/mvn help:evaluate -Dexpression=project.version - - build/mvn help:evaluate -Dexpression=java.version - - build/mvn help:evaluate -Dexpression=scala.binary.version - - build/mvn help:evaluate -Dexpression=hadoop.version - - build/mvn help:evaluate -Dexpression=hive.version + - mvn help:evaluate -Dexpression=project.version + - mvn help:evaluate -Dexpression=java.version + - mvn help:evaluate -Dexpression=scala.binary.version + - mvn help:evaluate -Dexpression=hadoop.version + - mvn help:evaluate -Dexpression=hive.version #jobs: # include: @@ -61,7 +61,7 @@ before_script: # - build/mvn --no-transfer-progress clean verify -pl :kyuubi-common,:kyuubi-ha,:kyuubi-main,:kyuubi-spark-sql-engine,:kyuubi-codecov,:kyuubi-download,:kyuubi-assembly -Dmaven.javadoc.skip=true -B -V script: - - build/mvn clean install -Dmaven.javadoc.skip=true -B -V + - mvn clean install -Dmaven.javadoc.skip=true -V after_script: - rm -r $HOME/.m2/repository/org/apache/kyuubi diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/SparkSQLEngineListenerSuite.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/SparkSQLEngineListenerSuite.scala index ba4ecf39d..e491b56dc 100644 --- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/SparkSQLEngineListenerSuite.scala +++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/SparkSQLEngineListenerSuite.scala @@ -17,27 +17,11 @@ package org.apache.kyuubi.engine.spark -import org.apache.spark.sql.SparkSession - -import org.apache.kyuubi.KyuubiFunSuite -import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.service.ServiceState -class SparkSQLEngineListenerSuite extends KyuubiFunSuite { - - override def beforeAll(): Unit = { - SparkSession.clearActiveSession() - SparkSession.clearDefaultSession() - super.beforeAll() - } +class SparkSQLEngineListenerSuite extends WithSparkSQLEngine { test("application end") { - val spark = SparkSession - .builder().master("local").config("spark.ui.port", "0").getOrCreate() - - val engine = new SparkSQLEngine(spark) - engine.initialize(KyuubiConf().set(KyuubiConf.FRONTEND_BIND_PORT, 0)) - engine.start() assert(engine.getServiceState === ServiceState.STARTED) spark.stop() assert(engine.getServiceState === ServiceState.STOPPED) diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/WithSparkSQLEngine.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/WithSparkSQLEngine.scala index ee9c17296..5bc4ce876 100644 --- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/WithSparkSQLEngine.scala +++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/WithSparkSQLEngine.scala @@ -51,10 +51,14 @@ trait WithSparkSQLEngine extends KyuubiFunSuite { super.afterAll() if (engine != null) { engine.stop() + engine = null } if (spark != null) { spark.stop() + spark = null } + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() SessionState.detachSession() Hive.closeCurrent() } diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/FrontendService.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/FrontendService.scala index 107f46513..d9f277b77 100644 --- a/kyuubi-common/src/main/scala/org/apache/kyuubi/service/FrontendService.scala +++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/service/FrontendService.scala @@ -19,7 +19,6 @@ package org.apache.kyuubi.service import java.net.{InetAddress, ServerSocket} import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -35,7 +34,7 @@ import org.apache.kyuubi.config.KyuubiConf import org.apache.kyuubi.operation.{FetchOrientation, OperationHandle} import org.apache.kyuubi.service.authentication.KyuubiAuthenticationFactory import org.apache.kyuubi.session.SessionHandle -import org.apache.kyuubi.util.{ExecutorPoolCaptureOom, KyuubiHadoopUtils} +import org.apache.kyuubi.util.{ExecutorPoolCaptureOom, KyuubiHadoopUtils, NamedThreadFactory} class FrontendService private (name: String, be: BackendService, oomHook: Runnable) extends AbstractService(name) with TCLIService.Iface with Runnable with Logging { @@ -49,9 +48,10 @@ class FrontendService private (name: String, be: BackendService, oomHook: Runnab } private var server: Option[TServer] = None + private var serverThread: Thread = _ protected var serverAddr: InetAddress = _ protected var portNum: Int = _ - protected var isStarted = new AtomicBoolean(false) + protected var isStarted = false private var authFactory: KyuubiAuthenticationFactory = _ private var hadoopConf: Configuration = _ @@ -114,10 +114,10 @@ class FrontendService private (name: String, be: BackendService, oomHook: Runnab override def start(): Unit = synchronized { super.start() - if (!isStarted.getAndSet(true)) { - val thread = new Thread(this) - thread.setName(getName) - thread.start() + if(!isStarted) { + serverThread = new NamedThreadFactory(getName, false).newThread(this) + serverThread.start() + isStarted = true } } @@ -125,15 +125,22 @@ class FrontendService private (name: String, be: BackendService, oomHook: Runnab info(s"Starting and exposing JDBC connection at: jdbc:hive2://$connectionUrl/") server.foreach(_.serve()) } catch { + case _: InterruptedException => error(s"$getName is interrupted") case t: Throwable => error(s"Error starting $getName", t) System.exit(-1) } override def stop(): Unit = synchronized { - if (isStarted.getAndSet(false)) { + if (isStarted) { + if (serverThread != null) { + serverThread.interrupt() + serverThread = null + } server.foreach(_.stop()) + server = None info(this.name + " has stopped") + isStarted = false } super.stop() }