[ https://issues.apache.org/jira/browse/SPARK-2390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Kousuke Saruta updated SPARK-2390: ---------------------------------- Affects Version/s: 1.0.1 > Files in staging directory cannot be deleted and wastes the space of HDFS > ------------------------------------------------------------------------- > > Key: SPARK-2390 > URL: https://issues.apache.org/jira/browse/SPARK-2390 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 1.0.0, 1.0.1 > Reporter: Kousuke Saruta > > When running jobs with YARN Cluster mode and using HistoryServer, the files > in the Staging Directory cannot be deleted. > HistoryServer uses directory where event log is written, and the directory is > represented as a instance of o.a.h.f.FileSystem created by using > FileSystem.get. > {code:title=FileLogger.scala} > private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) > {code} > {code:title=utils.getHadoopFileSystem} > def getHadoopFileSystem(path: URI): FileSystem = { > FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) > } > {code} > On the other hand, ApplicationMaster has a instance named fs, which also > created by using FileSystem.get. > {code:title=ApplicationMaster} > private val fs = FileSystem.get(yarnConf) > {code} > FileSystem.get returns cached same instance when URI passed to the method > represents same file system and the method is called by same user. > Because of the behavior, when the directory for event log is on HDFS, fs of > ApplicationMaster and fileSystem of FileLogger is same instance. > When shutting down ApplicationMaster, fileSystem.close is called in > FileLogger#stop, which is invoked by SparkContext#stop indirectly. > {code:title=FileLogger.stop} > def stop() { > hadoopDataStream.foreach(_.close()) > writer.foreach(_.close()) > fileSystem.close() > } > {code} > And ApplicationMaster#cleanupStagingDir also called by JVM shutdown hook. In > this method, fs.delete(stagingDirPath) is invoked. > Because fs.delete in ApplicationMaster is called after fileSystem.close in > FileLogger, fs.delete fails and results not deleting files in the staging > directory. -- This message was sent by Atlassian JIRA (v6.2#6252)