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
13 changes: 13 additions & 0 deletions core/src/main/scala/org/apache/spark/SparkEnv.scala
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,19 @@ 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.
if (SparkContext.DRIVER_IDENTIFIER == executorId) {
Copy link
Member

Choose a reason for hiding this comment

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

I see that this condition should indeed match the logic which controls whether this directory is a temp dir, but since the consequence of getting this wrong is pretty significant, I wonder if we should be more conservative. For example, save off the isDriver flag and use it in both places to be sure that these are exactly in sync? or maintain a separate reference to the temp dir that was created, which is None if no temp dir was created, and so you can be sure you can delete it if it's not None?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@srowen sorry, I don't very clear. You mean we can not use the executorId to distinguish the driver and executor?

Copy link
Member

Choose a reason for hiding this comment

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

I mean that this is not exactly the same condition that decided whether to make a temp dir, although, it looks like it will end up being the same. Since the consequence of accidentally getting this wrong or due to a later change, I suggest making this much more intimately bound. For example: save a reference to the temp dir if it exists. That you know you can delete.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@srowen Than you. If we want to make this much more intimately bound, may be we can check the sparkFilesDir directly, or else, we need to add a parameter to SparkEnv class.

Copy link
Member

Choose a reason for hiding this comment

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

I think you just need to store one more member, which is a reference to the tmp dir if it was created. See above. That's pretty much fool-proof and doesn't require any new params anywhere.

try {
Utils.deleteRecursively(new File(sparkFilesDir))
} catch {
case e: Exception =>
logWarning(s"Exception while deleting Spark temp dir: $sparkFilesDir", e)
}
}
}

private[spark]
Expand Down