From b38e0f04384860d2f32adb969407a3b8d184c2f7 Mon Sep 17 00:00:00 2001 From: Sephiroth-Lin Date: Sat, 7 Feb 2015 09:48:26 +0800 Subject: [PATCH] add dir check before delete 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. --- .../scala/org/apache/spark/SparkEnv.scala | 20 +++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index b71e289504f97..293e2b878455e 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -94,12 +94,20 @@ class SparkEnv ( // 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 - try { - Utils.deleteRecursively(new File(sparkFilesDir)) - } catch { - case e: Exception => logError(s"Exception while deleting Spark temp dir: $sparkFilesDir", e) + /** + * 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, so we need to check the sparkFilesDir, + * because sparkFilesDir is point to the current working dir in executor. + */ + if("." != sparkFilesDir){ + try { + Utils.deleteRecursively(new File(sparkFilesDir)) + } catch { + case e: Exception => + logWarning(s"Exception while deleting Spark temp dir: $sparkFilesDir", e) + } } }