[SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS#1326
[SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS#1326sarutak wants to merge 1 commit intoapache:masterfrom
Conversation
…er FileSystem operation
|
Caused by this issue, we can see IOException at Driver's(Application Maser's) log like as follows. 14/07/09 18:20:15 INFO spark.MapOutputTrackerMasterActor: MapOutputTrackerActor stopped! |
|
@sarutak This looks reasonable, though it doesn't seem that we close the |
|
Jenkins, test this please |
|
QA tests have started for PR 1326. This patch merges cleanly. |
|
QA results for PR 1326: |
|
LGTM |
|
Sure - I guess we can do this. It seems strange to open a filesystem and never close it (what if someone creates a large number of FileLogger instances... after all this is a generic class). I guess we'll rely on shutdown to close this. |
…the space of HDFS When running jobs with YARN Cluster mode and using HistoryServer, the files in the Staging Directory (~/.sparkStaging on HDFS) 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. On the other hand, ApplicationMaster has a instance named fs, which also created by using FileSystem.get. 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. 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. I think, calling fileSystem.delete is not needed. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes apache#1326 from sarutak/SPARK-2390 and squashes the following commits: 10e1a88 [Kousuke Saruta] Removed fileSystem.close from FileLogger.scala not to prevent any other FileSystem operation
…the space of HDFS When running jobs with YARN Cluster mode and using HistoryServer, the files in the Staging Directory (~/.sparkStaging on HDFS) 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. On the other hand, ApplicationMaster has a instance named fs, which also created by using FileSystem.get. 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. 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. I think, calling fileSystem.delete is not needed. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes apache#1326 from sarutak/SPARK-2390 and squashes the following commits: 10e1a88 [Kousuke Saruta] Removed fileSystem.close from FileLogger.scala not to prevent any other FileSystem operation
When running jobs with YARN Cluster mode and using HistoryServer, the files in the Staging Directory (~/.sparkStaging on HDFS) 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.
On the other hand, ApplicationMaster has a instance named fs, which also created by using FileSystem.get.
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.
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.
I think, calling fileSystem.delete is not needed.