From aeac51861e5c5bb185b0738dba2d25e57752038c Mon Sep 17 00:00:00 2001 From: Sephiroth-Lin Date: Fri, 6 Feb 2015 09:35:33 +0800 Subject: [PATCH] Delete tmp dir when sc is stop 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 --- core/src/main/scala/org/apache/spark/SparkEnv.scala | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index f25db7f8de565..a0836a2a68313 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -93,6 +93,14 @@ class SparkEnv ( // actorSystem.awaitTermination() // Note that blockTransferService is stopped by BlockManager since it is started by it. + + // If we only stop sc, but sparksubmit still run as a services we need to delete the tmp dir + // if not, it will create too many tmp dir + try { + Utils.deleteRecursively(new File(sparkFilesDir)) + } catch { + case e: Exception => logError(s"Exception while deleting Spark temp dir: $sparkFilesDir", e) + } } private[spark]