Skip to content

Commit

Permalink
Delete tmp dir when sc is stop
Browse files Browse the repository at this point in the history
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
  • Loading branch information
Sephiroth-Lin committed Feb 6, 2015
1 parent 85ccee8 commit c0d5b28
Showing 1 changed file with 9 additions and 0 deletions.
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 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(baseDir)
} catch {
case e: Exception =>
logError("Exception while deleting Spark temp dir: " + baseDir.getAbsolutePath, e)
}
}

def addFile(file: File) : String = {
Expand Down

0 comments on commit c0d5b28

Please sign in to comment.