Skip to content

[SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS#1326

Closed
sarutak wants to merge 1 commit intoapache:masterfrom
sarutak:SPARK-2390
Closed

[SPARK-2390] Files in staging directory cannot be deleted and wastes the space of HDFS#1326
sarutak wants to merge 1 commit intoapache:masterfrom
sarutak:SPARK-2390

Conversation

@sarutak
Copy link
Member

@sarutak sarutak commented Jul 8, 2014

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.

@sarutak
Copy link
Member Author

sarutak commented Jul 10, 2014

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!
14/07/09 18:20:15 INFO network.ConnectionManager: Selector thread was interrupted!
14/07/09 18:20:15 INFO network.ConnectionManager: ConnectionManager stopped
14/07/09 18:20:15 INFO storage.MemoryStore: MemoryStore cleared
14/07/09 18:20:15 INFO storage.BlockManager: BlockManager stopped
14/07/09 18:20:15 INFO storage.BlockManagerMasterActor: Stopping BlockManagerMaster
14/07/09 18:20:15 INFO storage.BlockManagerMaster: BlockManagerMaster stopped
14/07/09 18:20:15 INFO remote.RemoteActorRefProvider$RemotingTerminator: Shutting down remote daemon.
14/07/09 18:20:15 INFO remote.RemoteActorRefProvider$RemotingTerminator: Remote daemon shut down; proceeding with flushing remote transports.
14/07/09 18:20:15 INFO spark.SparkContext: Successfully stopped SparkContext
14/07/09 18:20:15 INFO yarn.ApplicationMaster: Unregistering ApplicationMaster with SUCCEEDED
14/07/09 18:20:15 INFO impl.AMRMClientImpl: Waiting for application to be successfully unregistered.
14/07/09 18:20:15 INFO Remoting: Remoting shut down
14/07/09 18:20:15 INFO remote.RemoteActorRefProvider$RemotingTerminator: Remoting shut down.
14/07/09 18:20:15 INFO yarn.ApplicationMaster$$anon$1: Invoking sc stop from shutdown hook
14/07/09 18:20:15 INFO ui.SparkUI: Stopped Spark web UI at http://spark-slave01:37382
14/07/09 18:20:15 INFO yarn.ApplicationMaster: AppMaster received a signal.
14/07/09 18:20:15 INFO yarn.ApplicationMaster: Deleting staging directory .sparkStaging/application_1404875428360_0024
14/07/09 18:20:15 ERROR yarn.ApplicationMaster: Failed to cleanup staging dir .sparkStaging/application_1404875428360_0024
java.io.IOException: Filesystem closed
at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:629)
at org.apache.hadoop.hdfs.DFSClient.delete(DFSClient.java:1618)
at org.apache.hadoop.hdfs.DistributedFileSystem$11.doCall(DistributedFileSystem.java:585)
at org.apache.hadoop.hdfs.DistributedFileSystem$11.doCall(DistributedFileSystem.java:581)
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:581)
at org.apache.spark.deploy.yarn.ApplicationMaster.org$apache$spark$deploy$yarn$ApplicationMaster$$cleanupStagingDir(ApplicationMaster.scala:345)
at org.apache.spark.deploy.yarn.ApplicationMaster$AppMasterShutdownHook.run(ApplicationMaster.scala:360)
at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)

@andrewor14
Copy link
Contributor

@sarutak This looks reasonable, though it doesn't seem that we close the FileSystem anywhere else in Spark. This is probably fine since we only call close() on exit anyway.

@andrewor14
Copy link
Contributor

Jenkins, test this please

@SparkQA
Copy link

SparkQA commented Jul 10, 2014

QA tests have started for PR 1326. This patch merges cleanly.
View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16522/consoleFull

@SparkQA
Copy link

SparkQA commented Jul 10, 2014

QA results for PR 1326:
- This patch PASSES unit tests.
- This patch merges cleanly
- This patch adds no public classes

For more information see test ouptut:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16522/consoleFull

@andrewor14
Copy link
Contributor

LGTM

@pwendell
Copy link
Contributor

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.

@asfgit asfgit closed this in c6d7574 Jul 15, 2014
gzm55 pushed a commit to MediaV/spark that referenced this pull request Jul 18, 2014
…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
xiliu82 pushed a commit to xiliu82/spark that referenced this pull request Sep 4, 2014
…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
@sarutak sarutak deleted the SPARK-2390 branch April 11, 2015 05:22
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants