[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) [&#10088;?&#10089;](https://pullrequestbadge.com/?utm_medium=github&utm_source=yaooqinn&utm_campaign=badge_info)<!-- PR-BADGE: PLEASE DO NOT REMOVE THIS COMMENT -->

<!--
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/yaooqinn/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.
-->
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 <yao@apache.org>
Co-authored-by: jhx1008 <jhx1008@gmail.com>
Signed-off-by: Kent Yao <yao@apache.org>
This commit is contained in:
Kent Yao 2021-02-02 20:46:06 +08:00
parent 6a4697019c
commit 7deb2f7fd7
No known key found for this signature in database
GPG Key ID: F7051850A0AF904D
5 changed files with 29 additions and 36 deletions

4
.gitignore vendored
View File

@ -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/

View File

@ -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

View File

@ -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)

View File

@ -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()
}

View File

@ -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()
}