Skip to content
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-5644] [Core]Delete tmp dir when sc is stop #4412

Closed
wants to merge 13 commits into from
9 changes: 9 additions & 0 deletions core/src/main/scala/org/apache/spark/HttpFileServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,15 @@ private[spark] class HttpFileServer(

def stop() {
httpServer.stop()

// 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
try {
Utils.deleteRecursively(baseDir)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems OK to me. I would make the error log a warning though as it's non-fatal. Consider using an interpolated string here as you do below.

} catch {
case e: Exception =>
logWarning("Exception while deleting Spark temp dir: " + baseDir.getAbsolutePath, e)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: consider interpolation here

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK.

}
}

def addFile(file: File) : String = {
Expand Down
27 changes: 26 additions & 1 deletion core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,8 @@ class SparkEnv (
// (e.g., HadoopFileRDD uses this to cache JobConfs and InputFormats).
private[spark] val hadoopJobMetadata = new MapMaker().softValues().makeMap[String, Any]()

private var tmpFilesDir: Option[String] = None
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah this looks good. Maybe call it driverTmpDirToDelete or something to be extra clear?


private[spark] def stop() {
isStopped = true
pythonWorkers.foreach { case(key, worker) => worker.stop() }
Expand All @@ -93,6 +95,22 @@ class SparkEnv (
// 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.
tmpFilesDir 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
}
}

private[spark]
Expand Down Expand Up @@ -350,7 +368,7 @@ object SparkEnv extends Logging {
"levels using the RDD.persist() method instead.")
}

new SparkEnv(
val envInstance = new SparkEnv(
executorId,
actorSystem,
serializer,
Expand All @@ -367,6 +385,13 @@ object SparkEnv extends Logging {
metricsSystem,
shuffleMemoryManager,
conf)

// Add a reference to tmp dir created by driver
if (isDriver) {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Might call out why this is linked to the isDriver logic above for future readers. @JoshRosen thoughts now?

envInstance.tmpFilesDir = Some(sparkFilesDir)
}

envInstance
}

/**
Expand Down