Details
-
Bug
-
Status: Resolved
-
Minor
-
Resolution: Won't Fix
-
1.5.2, 1.6.1, 2.0.0
-
None
-
None
Description
When running Spark (yarn,cluster mode) and killing application
.sparkStaging is not cleaned.
Reproduce:
1. run SparkPi job in yarn cluster mode
2. Wait app to switch to RUNNING and press Ctrl+C
3. kill app: $ yarn application -kill <app id>
4. Verify that it is not cleaned up: $ hadoop fs -ls .sparkStaging
Tested in Spark 1.5.2, 1.6.1, 2.0.0 - same error.
All of these apps are already finished/killed, but sparkStaging/application_<id> remains:
$ hadoop fs -ls .sparkStaging Found 6 items drwx------ - user user 3 2016-08-26 00:57 .sparkStaging/application_1472140614688_0001 drwx------ - user user 3 2016-08-26 01:09 .sparkStaging/application_1472140614688_0002 drwx------ - user user 3 2016-08-26 19:51 .sparkStaging/application_1472140614688_0005 drwx------ - user user 3 2016-08-26 19:53 .sparkStaging/application_1472140614688_0007 drwx------ - user user 3 2016-08-31 22:43 .sparkStaging/application_1472634296300_0011 drwx------ - user user 3 2016-08-31 23:30 .sparkStaging/application_1472651370711_0006
ERROR yarn.ApplicationMaster: RECEIVED SIGNAL 15: SIGTERM 16/08/26 00:51:17 INFO spark.SparkContext: Invoking stop() from shutdown hook 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 506.0 in stage 0.0 (TID 506, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 507.0 in stage 0.0 (TID 507, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 504.0 in stage 0.0 (TID 504) in 14 ms on node1 (505/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 505.0 in stage 0.0 (TID 505) in 14 ms on node1 (506/1000) 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/metrics/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/stage/kill,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/api,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/static,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/executors/threadDump/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/executors/threadDump,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/executors/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/executors,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/environment/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/environment,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/storage/rdd/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/storage/rdd,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/storage/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/storage,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/pool/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/pool,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/stage/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/stage,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/stages,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/jobs/job/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/jobs/job,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/jobs/json,null} 16/08/26 00:51:17 INFO handler.ContextHandler: stopped o.s.j.s.ServletContextHandler{/jobs,null} 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 508.0 in stage 0.0 (TID 508, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 507.0 in stage 0.0 (TID 507) in 20 ms on node1 (507/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 509.0 in stage 0.0 (TID 509, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 506.0 in stage 0.0 (TID 506) in 38 ms on node1 (508/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 510.0 in stage 0.0 (TID 510, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 508.0 in stage 0.0 (TID 508) in 35 ms on node1 (509/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 511.0 in stage 0.0 (TID 511, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 510.0 in stage 0.0 (TID 510) in 11 ms on node1 (510/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 512.0 in stage 0.0 (TID 512, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 509.0 in stage 0.0 (TID 509) in 25 ms on node1 (511/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 513.0 in stage 0.0 (TID 513, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 511.0 in stage 0.0 (TID 511) in 22 ms on node1 (512/1000) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 514.0 in stage 0.0 (TID 514, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 512.0 in stage 0.0 (TID 512) in 26 ms on node1 (513/1000) 16/08/26 00:51:17 INFO ui.SparkUI: Stopped Spark web UI at http://192.168.33.11:41489 16/08/26 00:51:17 INFO scheduler.DAGScheduler: Stopping DAGScheduler 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Starting task 515.0 in stage 0.0 (TID 515, node1, PROCESS_LOCAL, 2085 bytes) 16/08/26 00:51:17 INFO scheduler.DAGScheduler: ResultStage 0 (reduce at SparkPi.scala:36) failed in 7.827 s 16/08/26 00:51:17 INFO scheduler.DAGScheduler: Job 0 failed: reduce at SparkPi.scala:36, took 8.039526 s 16/08/26 00:51:17 INFO cluster.YarnClusterSchedulerBackend: Shutting down all executors 16/08/26 00:51:17 INFO scheduler.TaskSetManager: Finished task 514.0 in stage 0.0 (TID 514) in 21 ms on node1 (514/1000) 16/08/26 00:51:17 ERROR yarn.ApplicationMaster: User class threw exception: org.apache.spark.SparkException: Job cancelled because SparkContext was shut down org.apache.spark.SparkException: Job cancelled because SparkContext was shut down at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:703) at org.apache.spark.scheduler.DAGScheduler$$anonfun$cleanUpAfterSchedulerStop$1.apply(DAGScheduler.scala:702) at scala.collection.mutable.HashSet.foreach(HashSet.scala:79) at org.apache.spark.scheduler.DAGScheduler.cleanUpAfterSchedulerStop(DAGScheduler.scala:702) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onStop(DAGScheduler.scala:1514) at org.apache.spark.util.EventLoop.stop(EventLoop.scala:84) at org.apache.spark.scheduler.DAGScheduler.stop(DAGScheduler.scala:1438) at org.apache.spark.SparkContext$$anonfun$stop$7.apply$mcV$sp(SparkContext.scala:1724) at org.apache.spark.util.Utils$.tryLogNonFatalError(Utils.scala:1185) at org.apache.spark.SparkContext.stop(SparkContext.scala:1723) at org.apache.spark.SparkContext$$anonfun$3.apply$mcV$sp(SparkContext.scala:587) at org.apache.spark.util.SparkShutdownHook.run(ShutdownHookManager.scala:264) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ShutdownHookManager.scala:234) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:234) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1$$anonfun$apply$mcV$sp$1.apply(ShutdownHookManager.scala:234) at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1699) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply$mcV$sp(ShutdownHookManager.scala:234) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:234) at org.apache.spark.util.SparkShutdownHookManager$$anonfun$runAll$1.apply(ShutdownHookManager.scala:234) at scala.util.Try$.apply(Try.scala:161) at org.apache.spark.util.SparkShutdownHookManager.runAll(ShutdownHookManager.scala:234) at org.apache.spark.util.SparkShutdownHookManager$$anon$2.run(ShutdownHookManager.scala:216) at org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:567) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1824) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1944) at org.apache.spark.rdd.RDD$$anonfun$reduce$1.apply(RDD.scala:1007) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:108) at org.apache.spark.rdd.RDD.withScope(RDD.scala:310) at org.apache.spark.rdd.RDD.reduce(RDD.scala:989) at org.apache.spark.examples.SparkPi$.main(SparkPi.scala:36) at org.apache.spark.examples.SparkPi.main(SparkPi.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:525) 16/08/26 00:51:17 INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 15, (reason: User class threw exception: org.apache.spark.SparkException: Job cancelled because SparkContext was shut down) 16/08/26 00:51:17 INFO cluster.YarnClusterSchedulerBackend: Asking each executor to shut down End of LogType:stderr
Currently cleanupStagingDir is run when app has final status SUCCEEDED.
if (!unregistered) { // we only want to unregister if we don't want the RM to retry if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { unregister(finalStatus, finalMsg) cleanupStagingDir(fs) } }
If we kill spark-submit in terminal with Ctrl+C - the cleanup is not performed.
Can we do unregister and cleanup if we get
(yarn kill)
exitCode 15 in Spark 1.5.X/1.6.X
ERROR yarn.ApplicationMaster: RECEIVED SIGNAL 15: SIGTERM
INFO yarn.ApplicationMaster: Final app status: FAILED, exitCode: 15, (reason: User class threw exception: org.apache.spark.SparkException: Job cancelled because SparkContext was shut down)
and exitCode 16 in Spark 2.0.X
ERROR ApplicationMaster: RECEIVED SIGNAL TERM
INFO ApplicationMaster: Final app status: FAILED, exitCode: 16, (reason: Shutdown hook called before final status was reported.)