Are you running a vanilla Hadoop 2.3.0 or the one that comes with CDH5 /
HDP(?) ? We may be able to reproduce this in that case.

TD

On Wed, May 21, 2014 at 8:35 PM, Tom Graves <tgraves...@yahoo.com> wrote:

> It sounds like something is closing the hdfs filesystem before everyone is
> really done with it. The filesystem gets cached and is shared so if someone
> closes it while other threads are still using it you run into this error.   Is
> your application closing the filesystem?     Are you using the event
> logging feature?   Could you share the options you are running with?
>
> Yarn will retry the application depending on how the Application Master
> attempt fails (this is a configurable setting as to how many times it
> retries).  That is probably the second driver you are referring to.  But
> they shouldn't have overlapped as far as both being up at the same time. Is
> that the case you are seeing?  Generally you want to look at why the first
> application attempt fails.
>
> Tom
>
>
>
>   On Wednesday, May 21, 2014 6:10 PM, Kevin Markey <
> kevin.mar...@oracle.com> wrote:
>
>
>  I tested an application on RC-10 and Hadoop 2.3.0 in yarn-cluster mode
> that had run successfully with Spark-0.9.1 and Hadoop 2.3 or 2.2.  The
> application successfully ran to conclusion but it ultimately failed.
>
> There were 2 anomalies...
>
> 1. ASM reported only that the application was "ACCEPTED".  It never
> indicated that the application was "RUNNING."
>
> 14/05/21 16:06:12 INFO yarn.Client: Application report from ASM:
>      application identifier: application_1400696988985_0007
>      appId: 7
>      clientToAMToken: null
>      appDiagnostics:
>      appMasterHost: N/A
>      appQueue: default
>      appMasterRpcPort: -1
>      appStartTime: 1400709970857
>      yarnAppState: ACCEPTED
>      distributedFinalState: UNDEFINED
>      appTrackingUrl:
> http://Sleepycat:8088/proxy/application_1400696988985_0007/<http://sleepycat:8088/proxy/application_1400696988985_0007/>
>      appUser: hduser
>
> Furthermore, it *started a second container*, running two partly
> *overlapping* drivers, when it appeared that the application never
> started.  Each container ran to conclusion as explained above, taking twice
> as long as usual for both to complete.  Both instances had the same
> concluding failure.
>
> 2. Each instance failed as indicated by the stderr log, finding that the 
> *filesystem
> was closed* when trying to clean up the staging directories.
>
> 14/05/21 16:08:24 INFO Executor: Serialized size of result for 1453 is 863
> 14/05/21 16:08:24 INFO Executor: Sending result for 1453 directly to driver
> 14/05/21 16:08:24 INFO Executor: Finished task ID 1453
> 14/05/21 16:08:24 INFO TaskSetManager: Finished TID 1453 in 202 ms on
> localhost (progress: 2/2)
> 14/05/21 16:08:24 INFO DAGScheduler: Completed ResultTask(1507, 1)
> 14/05/21 16:08:24 INFO TaskSchedulerImpl: Removed TaskSet 1507.0, whose
> tasks have all completed, from pool
> 14/05/21 16:08:24 INFO DAGScheduler: Stage 1507 (count at KEval.scala:32)
> finished in 0.417 s
> 14/05/21 16:08:24 INFO SparkContext: Job finished: count at
> KEval.scala:32, took 1.532789283 s
> 14/05/21 16:08:24 INFO SparkUI: Stopped Spark web UI at
> http://dhcp-brm-bl1-215-1e-east-10-135-123-92.usdhcp.oraclecorp.com:42250
> 14/05/21 16:08:24 INFO DAGScheduler: Stopping DAGScheduler
> 14/05/21 16:08:25 INFO MapOutputTrackerMasterActor: MapOutputTrackerActor
> stopped!
> 14/05/21 16:08:25 INFO ConnectionManager: Selector thread was interrupted!
> 14/05/21 16:08:25 INFO ConnectionManager: ConnectionManager stopped
> 14/05/21 16:08:25 INFO MemoryStore: MemoryStore cleared
> 14/05/21 16:08:25 INFO BlockManager: BlockManager stopped
> 14/05/21 16:08:25 INFO BlockManagerMasterActor: Stopping BlockManagerMaster
> 14/05/21 16:08:25 INFO BlockManagerMaster: BlockManagerMaster stopped
> 14/05/21 16:08:25 INFO SparkContext: Successfully stopped SparkContext
> 14/05/21 16:08:25 INFO RemoteActorRefProvider$RemotingTerminator: Shutting
> down remote daemon.
> 14/05/21 16:08:25 INFO ApplicationMaster: *finishApplicationMaster with
> SUCCEEDED*
> 14/05/21 16:08:25 INFO ApplicationMaster: AppMaster received a signal.
> 14/05/21 16:08:25 INFO ApplicationMaster: Deleting staging directory
> .sparkStaging/application_1400696988985_0007
> 14/05/21 16:08:25 INFO RemoteActorRefProvider$RemotingTerminator: Remote
> daemon shut down; proceeding with flushing remote transports.
> 14/05/21 16:08:25 ERROR *ApplicationMaster: Failed to cleanup staging dir
> .sparkStaging/application_1400696988985_0007*
> *java.io.IOException: Filesystem closed*
>     at org.apache.hadoop.hdfs.DFSClient.checkOpen(DFSClient.java:689)
>     at org.apache.hadoop.hdfs.DFSClient.delete(DFSClient.java:1685)
>     at
> org.apache.hadoop.hdfs.DistributedFileSystem$11.doCall(DistributedFileSystem.java:591)
>     at
> org.apache.hadoop.hdfs.DistributedFileSystem$11.doCall(DistributedFileSystem.java:587)
>     at
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>     at
> org.apache.hadoop.hdfs.DistributedFileSystem.delete(DistributedFileSystem.java:587)
>     at org.apache.spark.deploy.yarn.ApplicationMaster.org
> $apache$spark$deploy$yarn$ApplicationMaster$$cleanupStagingDir(ApplicationMaster.scala:371)
>     at
> org.apache.spark.deploy.yarn.ApplicationMaster$AppMasterShutdownHook.run(ApplicationMaster.scala:386)
>     at
> org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)
>
> There is nothing about the staging directory themselves that looks
> suspicious...
>
> drwx------   - hduser supergroup          0 2014-05-21 16:06
> /user/hduser/.sparkStaging/application_1400696988985_0007
> -rw-r--r--   3 hduser supergroup   92881278 2014-05-21 16:06
> /user/hduser/.sparkStaging/application_1400696988985_0007/app.jar
> -rw-r--r--   3 hduser supergroup  118900783 2014-05-21 16:06
> /user/hduser/.sparkStaging/application_1400696988985_0007/spark-assembly-1.0.0-hadoop2.3.0.jar
>
> Just prior to the staging directory cleanup, the application concluded by
> writing results to 3 HDFS files.  That occurred without incident.
>
> This particular test was run using ...
>
> 1. RC10 compiled as follows:  *mvn -Pyarn -Phadoop-2.3
> -Dhadoop.version=2.3.0 -DskipTests clean package*
> 2. Ran in yarn-cluster mode using spark-submit
>
> Is there any configuration new to 1.0.0 that I might be missing.  I walked
> through all the changes in the Yarn deploy web page, updating my scripts
> and configuration appropriately, and running except for these two anomalies.
>
> Thanks
> Kevin Markey
>
>
>
>
>
>

Reply via email to