Skip to content

Commit f590178

Browse files
tedyuAndrew Or
authored and
Andrew Or
committed
[SPARK-12365][CORE] Use ShutdownHookManager where Runtime.getRuntime.addShutdownHook() is called
SPARK-9886 fixed ExternalBlockStore.scala This PR fixes the remaining references to Runtime.getRuntime.addShutdownHook() Author: tedyu <[email protected]> Closes #10325 from ted-yu/master.
1 parent 38d9795 commit f590178

File tree

5 files changed

+38
-33
lines changed

5 files changed

+38
-33
lines changed

core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala

+6-12
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ import org.apache.spark.network.sasl.SaslServerBootstrap
2828
import org.apache.spark.network.server.{TransportServerBootstrap, TransportServer}
2929
import org.apache.spark.network.shuffle.ExternalShuffleBlockHandler
3030
import org.apache.spark.network.util.TransportConf
31-
import org.apache.spark.util.Utils
31+
import org.apache.spark.util.{ShutdownHookManager, Utils}
3232

3333
/**
3434
* Provides a server from which Executors can read shuffle files (rather than reading directly from
@@ -118,19 +118,13 @@ object ExternalShuffleService extends Logging {
118118
server = newShuffleService(sparkConf, securityManager)
119119
server.start()
120120

121-
installShutdownHook()
121+
ShutdownHookManager.addShutdownHook { () =>
122+
logInfo("Shutting down shuffle service.")
123+
server.stop()
124+
barrier.countDown()
125+
}
122126

123127
// keep running until the process is terminated
124128
barrier.await()
125129
}
126-
127-
private def installShutdownHook(): Unit = {
128-
Runtime.getRuntime.addShutdownHook(new Thread("External Shuffle Service shutdown thread") {
129-
override def run() {
130-
logInfo("Shutting down shuffle service.")
131-
server.stop()
132-
barrier.countDown()
133-
}
134-
})
135-
}
136130
}

core/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala

+5-8
Original file line numberDiff line numberDiff line change
@@ -22,7 +22,7 @@ import java.util.concurrent.CountDownLatch
2222
import org.apache.spark.deploy.mesos.ui.MesosClusterUI
2323
import org.apache.spark.deploy.rest.mesos.MesosRestServer
2424
import org.apache.spark.scheduler.cluster.mesos._
25-
import org.apache.spark.util.SignalLogger
25+
import org.apache.spark.util.{ShutdownHookManager, SignalLogger}
2626
import org.apache.spark.{Logging, SecurityManager, SparkConf}
2727

2828
/*
@@ -103,14 +103,11 @@ private[mesos] object MesosClusterDispatcher extends Logging {
103103
}
104104
val dispatcher = new MesosClusterDispatcher(dispatcherArgs, conf)
105105
dispatcher.start()
106-
val shutdownHook = new Thread() {
107-
override def run() {
108-
logInfo("Shutdown hook is shutting down dispatcher")
109-
dispatcher.stop()
110-
dispatcher.awaitShutdown()
111-
}
106+
ShutdownHookManager.addShutdownHook { () =>
107+
logInfo("Shutdown hook is shutting down dispatcher")
108+
dispatcher.stop()
109+
dispatcher.awaitShutdown()
112110
}
113-
Runtime.getRuntime.addShutdownHook(shutdownHook)
114111
dispatcher.awaitShutdown()
115112
}
116113
}

core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala

+4
Original file line numberDiff line numberDiff line change
@@ -162,7 +162,9 @@ private[spark] object ShutdownHookManager extends Logging {
162162
val hook = new Thread {
163163
override def run() {}
164164
}
165+
// scalastyle:off runtimeaddshutdownhook
165166
Runtime.getRuntime.addShutdownHook(hook)
167+
// scalastyle:on runtimeaddshutdownhook
166168
Runtime.getRuntime.removeShutdownHook(hook)
167169
} catch {
168170
case ise: IllegalStateException => return true
@@ -228,7 +230,9 @@ private [util] class SparkShutdownHookManager {
228230
.invoke(shm, hookTask, Integer.valueOf(fsPriority + 30))
229231

230232
case Failure(_) =>
233+
// scalastyle:off runtimeaddshutdownhook
231234
Runtime.getRuntime.addShutdownHook(new Thread(hookTask, "Spark Shutdown Hook"));
235+
// scalastyle:on runtimeaddshutdownhook
232236
}
233237
}
234238

scalastyle-config.xml

+12
Original file line numberDiff line numberDiff line change
@@ -157,6 +157,18 @@ This file is divided into 3 sections:
157157
]]></customMessage>
158158
</check>
159159

160+
<check customId="runtimeaddshutdownhook" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
161+
<parameters><parameter name="regex">Runtime\.getRuntime\.addShutdownHook</parameter></parameters>
162+
<customMessage><![CDATA[
163+
Are you sure that you want to use Runtime.getRuntime.addShutdownHook? In most cases, you should use
164+
ShutdownHookManager.addShutdownHook instead.
165+
If you must use Runtime.getRuntime.addShutdownHook, wrap the code block with
166+
// scalastyle:off runtimeaddshutdownhook
167+
Runtime.getRuntime.addShutdownHook(...)
168+
// scalastyle:on runtimeaddshutdownhook
169+
]]></customMessage>
170+
</check>
171+
160172
<check customId="classforname" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
161173
<parameters><parameter name="regex">Class\.forName</parameter></parameters>
162174
<customMessage><![CDATA[

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala

+11-13
Original file line numberDiff line numberDiff line change
@@ -195,20 +195,18 @@ private[hive] object SparkSQLCLIDriver extends Logging {
195195
}
196196

197197
// add shutdown hook to flush the history to history file
198-
Runtime.getRuntime.addShutdownHook(new Thread(new Runnable() {
199-
override def run() = {
200-
reader.getHistory match {
201-
case h: FileHistory =>
202-
try {
203-
h.flush()
204-
} catch {
205-
case e: IOException =>
206-
logWarning("WARNING: Failed to write command history file: " + e.getMessage)
207-
}
208-
case _ =>
209-
}
198+
ShutdownHookManager.addShutdownHook { () =>
199+
reader.getHistory match {
200+
case h: FileHistory =>
201+
try {
202+
h.flush()
203+
} catch {
204+
case e: IOException =>
205+
logWarning("WARNING: Failed to write command history file: " + e.getMessage)
206+
}
207+
case _ =>
210208
}
211-
}))
209+
}
212210

213211
// TODO: missing
214212
/*

0 commit comments

Comments
 (0)