[ https://issues.apache.org/jira/browse/SPARK-19868?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Kay Ousterhout resolved SPARK-19868. ------------------------------------ Resolution: Fixed Fix Version/s: 2.2.0 > conflict TasksetManager lead to spark stopped > --------------------------------------------- > > Key: SPARK-19868 > URL: https://issues.apache.org/jira/browse/SPARK-19868 > Project: Spark > Issue Type: Bug > Components: Spark Core > Affects Versions: 2.1.0 > Reporter: liujianhui > Fix For: 2.2.0 > > > ##scenario > conflict taskSetManager throw an exception which lead to sparkcontext > stopped. log as > {code} > java.lang.IllegalStateException: more than one active taskSet for stage > 4571114: 4571114.2,4571114.1 > at > org.apache.spark.scheduler.TaskSchedulerImpl.submitTasks(TaskSchedulerImpl.scala:173) > at > org.apache.spark.scheduler.DAGScheduler.submitMissingTasks(DAGScheduler.scala:1052) > at > org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:921) > at > org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:1214) > at > org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1637) > at > org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1599) > at > org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1588) > at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) > {code} > the reason for that is the resubmitting of stage conflict with the running > stageļ¼the missing task of stage should be resubmit since the zoombie of the > tasksetManager assigned by true > {code} > [INFO][dag-scheduler-event-loop][2017-03-03+22:16:29.547][org.apache.spark.scheduler.DAGScheduler]Resubmitting > ShuffleMapStage 4571114 (map at MainApp.scala:73) because some of its tasks > had failed: 0 > [INFO][dag-scheduler-event-loop][2017-03-03+22:16:29.547][org.apache.spark.scheduler.DAGScheduler]Submitting > ShuffleMapStage 4571114 (MapPartitionsRDD[3719544] at map at > MainApp.scala:73), which has no missing parents > {code} > the executor which the shuffle task ran on was lost > {code} > [INFO][dag-scheduler-event-loop][2017-03-03+22:16:27.427][org.apache.spark.scheduler.DAGScheduler]Ignoring > possibly bogus ShuffleMapTask(4571114, 0) completion from executor 4 > {code} > the time of the task set finished and the resubmit of stage > {code} > handleSuccessfuleTask > [INFO][task-result-getter-2][2017-03-03+22:16:29.999][org.apache.spark.scheduler.TaskSchedulerImpl]Removed > TaskSet 4571114.1, whose tasks have all completed, from pool > resubmit stage > [INFO][dag-scheduler-event-loop][2017-03-03+22:16:29.549][org.apache.spark.scheduler.TaskSchedulerImpl]Adding > task set 4571114.2 with 1 tasks > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org