-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-2645] [Core] Allow SparkEnv.stop() to be called multiple times without side effects. #6973
Changes from 14 commits
e3677c9
106fd8e
0be142d
b566b66
380c5b0
58dba70
9193a0c
a5a7d7f
72bb484
12f66b5
1aff39c
c97839a
2ce5760
446b0a4
277043e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -45,6 +45,8 @@ import org.apache.spark.storage._ | |
import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, MemoryAllocator} | ||
import org.apache.spark.util.{RpcUtils, Utils} | ||
|
||
import scala.util.control.NonFatal | ||
|
||
/** | ||
* :: DeveloperApi :: | ||
* Holds all the runtime environment objects for a running Spark instance (either master or worker), | ||
|
@@ -90,39 +92,44 @@ class SparkEnv ( | |
private var driverTmpDirToDelete: Option[String] = None | ||
|
||
private[spark] def stop() { | ||
isStopped = true | ||
pythonWorkers.foreach { case(key, worker) => worker.stop() } | ||
Option(httpFileServer).foreach(_.stop()) | ||
mapOutputTracker.stop() | ||
shuffleManager.stop() | ||
broadcastManager.stop() | ||
blockManager.stop() | ||
blockManager.master.stop() | ||
metricsSystem.stop() | ||
outputCommitCoordinator.stop() | ||
rpcEnv.shutdown() | ||
|
||
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut | ||
// down, but let's call it anyway in case it gets fixed in a later release | ||
// UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. | ||
// actorSystem.awaitTermination() | ||
|
||
// Note that blockTransferService is stopped by BlockManager since it is started by it. | ||
|
||
// If we only stop sc, but the driver process still run as a services then we need to delete | ||
// the tmp dir, if not, it will create too many tmp dirs. | ||
// We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the | ||
// current working dir in executor which we do not need to delete. | ||
driverTmpDirToDelete match { | ||
case Some(path) => { | ||
try { | ||
Utils.deleteRecursively(new File(path)) | ||
} catch { | ||
case e: Exception => | ||
logWarning(s"Exception while deleting Spark temp dir: $path", e) | ||
|
||
if (!isStopped) { | ||
isStopped = true | ||
|
||
pythonWorkers.foreach { case (key, worker) => worker.stop()} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Very minor, but if you're changing the file again, add a space before the brace at the end, and this can be There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. or just |
||
Option(httpFileServer).foreach(_.stop()) | ||
mapOutputTracker.stop() | ||
shuffleManager.stop() | ||
broadcastManager.stop() | ||
blockManager.stop() | ||
blockManager.master.stop() | ||
metricsSystem.stop() | ||
outputCommitCoordinator.stop() | ||
rpcEnv.shutdown() | ||
|
||
|
||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you kill this blank line |
||
// Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut | ||
// down, but let's call it anyway in case it gets fixed in a later release | ||
// UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. | ||
// actorSystem.awaitTermination() | ||
|
||
// Note that blockTransferService is stopped by BlockManager since it is started by it. | ||
|
||
// If we only stop sc, but the driver process still run as a services then we need to delete | ||
// the tmp dir, if not, it will create too many tmp dirs. | ||
// We only need to delete the tmp dir create by driver, because sparkFilesDir is point to the | ||
// current working dir in executor which we do not need to delete. | ||
driverTmpDirToDelete match { | ||
case Some(path) => { | ||
try { | ||
Utils.deleteRecursively(new File(path)) | ||
} catch { | ||
case e: Exception => | ||
logWarning(s"Exception while deleting Spark temp dir: $path", e) | ||
} | ||
} | ||
case None => // We just need to delete tmp dir created by driver, so do nothing on executor | ||
} | ||
case None => // We just need to delete tmp dir created by driver, so do nothing on executor | ||
} | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Not needed now?