Skip to content

Commit 3b903e4

Browse files
author
Andrew Or
committed
Revert "[SPARK-12365][CORE] Use ShutdownHookManager where Runtime.getRuntime.addShutdownHook() is called"
This reverts commit 4af6438.
1 parent 1dc71ec commit 3b903e4

File tree

4 files changed

+20
-27
lines changed

4 files changed

+20
-27
lines changed

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

Lines changed: 12 additions & 6 deletions
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.{ShutdownHookManager, Utils}
31+
import org.apache.spark.util.Utils
3232

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

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

127123
// keep running until the process is terminated
128124
barrier.await()
129125
}
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+
}
130136
}

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

Lines changed: 8 additions & 5 deletions
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.{ShutdownHookManager, SignalLogger}
25+
import org.apache.spark.util.SignalLogger
2626
import org.apache.spark.{Logging, SecurityManager, SparkConf}
2727

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

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

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -162,9 +162,7 @@ private[spark] object ShutdownHookManager extends Logging {
162162
val hook = new Thread {
163163
override def run() {}
164164
}
165-
// scalastyle:off runtimeaddshutdownhook
166165
Runtime.getRuntime.addShutdownHook(hook)
167-
// scalastyle:on runtimeaddshutdownhook
168166
Runtime.getRuntime.removeShutdownHook(hook)
169167
} catch {
170168
case ise: IllegalStateException => return true
@@ -230,9 +228,7 @@ private [util] class SparkShutdownHookManager {
230228
.invoke(shm, hookTask, Integer.valueOf(fsPriority + 30))
231229

232230
case Failure(_) =>
233-
// scalastyle:off runtimeaddshutdownhook
234231
Runtime.getRuntime.addShutdownHook(new Thread(hookTask, "Spark Shutdown Hook"));
235-
// scalastyle:on runtimeaddshutdownhook
236232
}
237233
}
238234

scalastyle-config.xml

Lines changed: 0 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -157,18 +157,6 @@ 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-
172160
<check customId="classforname" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
173161
<parameters><parameter name="regex">Class\.forName</parameter></parameters>
174162
<customMessage><![CDATA[

0 commit comments

Comments
 (0)