-
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-5644] [Core]Delete tmp dir when sc is stop #4412
Conversation
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit
You'll need a JIRA for this. I think it's probably a harmless change, but, a process contains one SparkContext, and isn't generally doing anything else but Spark, so what problem does this cause in practice? |
@srowen we run a process as a service which will not stop. In this service process we will create SparkContext and run job and then stop it, because we only call sc.stop but not exit this service process so the tmp dirs created by HttpFileServer and SparkEnv will not be deleted after SparkContext is stopped, and this will lead to creating too many tmp dirs if we create many SparkContext to run job in this service process. |
// 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) |
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.
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.
make the error log a warning though as it's non-fatal.
sparkFilesDir is point to the current working dir in executor, and we only need to delete the tmp dir create by driver, for safe, we check it first.
// 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) { |
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.
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
?
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.
@srowen sorry, I don't very clear. You mean we can not use the executorId to distinguish the driver and 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.
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.
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.
@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.
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.
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.
change from < if (SparkContext.DRIVER_IDENTIFIER == executorId) > to < if (sparkFilesDir != ".") >, and add comment where sparkFilesDir is created.
ok to test |
Test build #27097 has finished for PR 4412 at commit
|
@srowen thank you, now I add a member to store the reference of the tmp dir if it was created, please help to check again. |
Test build #27186 has finished for PR 4412 at commit
|
Utils.deleteRecursively(baseDir) | ||
} catch { | ||
case e: Exception => | ||
logWarning("Exception while deleting Spark temp dir: " + baseDir.getAbsolutePath, e) |
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.
Nit: consider interpolation here
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.
OK.
@srowen thank you, please help to check again. |
Test build #27199 has finished for PR 4412 at commit
|
Test build #27200 has finished for PR 4412 at commit
|
OK, that's looking good to me. I'll pause shortly to let others review it too. |
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly. Author: Sephiroth-Lin <[email protected]> Closes apache#4412 from Sephiroth-Lin/bug-fix-master-01 and squashes the following commits: fbbc785 [Sephiroth-Lin] using an interpolated string b968e14 [Sephiroth-Lin] using an interpolated string 4edf394 [Sephiroth-Lin] rename the variable and update comment 1339c96 [Sephiroth-Lin] add a member to store the reference of tmp dir b2018a5 [Sephiroth-Lin] check sparkFilesDir before delete f48a3c6 [Sephiroth-Lin] don't check sparkFilesDir, check executorId dd9686e [Sephiroth-Lin] format code b38e0f0 [Sephiroth-Lin] add dir check before delete d7ccc64 [Sephiroth-Lin] Change log level 1d70926 [Sephiroth-Lin] update comment e2a2b1b [Sephiroth-Lin] update comment aeac518 [Sephiroth-Lin] Delete tmp dir when sc is stop c0d5b28 [Sephiroth-Lin] Delete tmp dir when sc is stop Conflicts: core/src/main/scala/org/apache/spark/SparkEnv.scala
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly. Author: Sephiroth-Lin <[email protected]> Closes apache#4412 from Sephiroth-Lin/bug-fix-master-01 and squashes the following commits: fbbc785 [Sephiroth-Lin] using an interpolated string b968e14 [Sephiroth-Lin] using an interpolated string 4edf394 [Sephiroth-Lin] rename the variable and update comment 1339c96 [Sephiroth-Lin] add a member to store the reference of tmp dir b2018a5 [Sephiroth-Lin] check sparkFilesDir before delete f48a3c6 [Sephiroth-Lin] don't check sparkFilesDir, check executorId dd9686e [Sephiroth-Lin] format code b38e0f0 [Sephiroth-Lin] add dir check before delete d7ccc64 [Sephiroth-Lin] Change log level 1d70926 [Sephiroth-Lin] update comment e2a2b1b [Sephiroth-Lin] update comment aeac518 [Sephiroth-Lin] Delete tmp dir when sc is stop c0d5b28 [Sephiroth-Lin] Delete tmp dir when sc is stop Conflicts: core/src/main/scala/org/apache/spark/SparkEnv.scala
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly. Author: Sephiroth-Lin <[email protected]> Closes apache#4412 from Sephiroth-Lin/bug-fix-master-01 and squashes the following commits: fbbc785 [Sephiroth-Lin] using an interpolated string b968e14 [Sephiroth-Lin] using an interpolated string 4edf394 [Sephiroth-Lin] rename the variable and update comment 1339c96 [Sephiroth-Lin] add a member to store the reference of tmp dir b2018a5 [Sephiroth-Lin] check sparkFilesDir before delete f48a3c6 [Sephiroth-Lin] don't check sparkFilesDir, check executorId dd9686e [Sephiroth-Lin] format code b38e0f0 [Sephiroth-Lin] add dir check before delete d7ccc64 [Sephiroth-Lin] Change log level 1d70926 [Sephiroth-Lin] update comment e2a2b1b [Sephiroth-Lin] update comment aeac518 [Sephiroth-Lin] Delete tmp dir when sc is stop c0d5b28 [Sephiroth-Lin] Delete tmp dir when sc is stop Conflicts: core/src/main/scala/org/apache/spark/SparkEnv.scala
When we run driver as a service, and for each time we run job we only call sc.stop, then will not delete tmp dir create by HttpFileServer and SparkEnv, it will be deleted until the service process exit, so we need to delete these tmp dirs when sc is stop directly.