Skip to content

Commit

Permalink
SPARK-2645: Fix for SparkContext stop behavior
Browse files Browse the repository at this point in the history
  • Loading branch information
rekhajoshm committed Jun 24, 2015
1 parent 0be142d commit b566b66
Showing 1 changed file with 18 additions and 10 deletions.
28 changes: 18 additions & 10 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -90,17 +90,25 @@ class SparkEnv (
private var driverTmpDirToDelete: Option[String] = None

private[spark] def stop() {

if(isStopped) return

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()
try {
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()
} catch {
case e: Exception =>
logInfo("Exception while SparkEnv stop", e)
}

// 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
Expand Down

0 comments on commit b566b66

Please sign in to comment.