[
https://issues.apache.org/jira/browse/SPARK-23433?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16366417#comment-16366417
]
Shixiong Zhu commented on SPARK-23433:
--------------------------------------
{code}
18/02/11 13:22:20 INFO TaskSetManager: Finished task 17.0 in stage 7580621.1
(TID 65577139) in 303870 ms on 10.0.246.111 (executor 24) (18/19)
18/02/11 13:22:20 INFO DAGScheduler: ShuffleMapStage 7580621 (start at
command-2841337:340) finished in 303.880 s
18/02/11 13:22:20 INFO DAGScheduler: Resubmitting ShuffleMapStage 7580621
(start at command-2841337:340) because some of its tasks had failed: 2, 15, 27,
28, 41
18/02/11 13:22:27 INFO DAGScheduler: Submitting ShuffleMapStage 7580621
(MapPartitionsRDD[2660062] at start at command-2841337:340), which has no
missing parents
18/02/11 13:22:27 INFO DAGScheduler: Submitting 5 missing tasks from
ShuffleMapStage 7580621 (MapPartitionsRDD[2660062] at start at
command-2841337:340) (first 15 tasks are for partitions Vector(2, 15, 27, 28,
41))
18/02/11 13:22:27 INFO TaskSchedulerImpl: Adding task set 7580621.2 with 5 tasks
18/02/11 13:22:27 ERROR DAGSchedulerEventProcessLoop:
DAGSchedulerEventProcessLoop failed; shutting down SparkContext
java.lang.IllegalStateException: more than one active taskSet for stage
7580621: 7580621.2,7580621.1
at
org.apache.spark.scheduler.TaskSchedulerImpl.submitTasks(TaskSchedulerImpl.scala:229)
at
org.apache.spark.scheduler.DAGScheduler.submitMissingTasks(DAGScheduler.scala:1193)
at
org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:1059)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:900)
at
org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:899)
at
scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
at
org.apache.spark.scheduler.DAGScheduler.submitWaitingChildStages(DAGScheduler.scala:899)
at
org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:1427)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1929)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1880)
at
org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1868)
at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
18/02/11 13:22:27 INFO TaskSchedulerImpl: Cancelling stage 7580621
18/02/11 13:22:27 INFO TaskSchedulerImpl: Cancelling stage 7580621
18/02/11 13:22:27 INFO TaskSchedulerImpl: Stage 7580621 was cancelled
18/02/11 13:22:27 INFO DAGScheduler: ShuffleMapStage 7580621 (start at
command-2841337:340) failed in 0.057 s due to Job aborted due to stage failure:
Stage 7580621 cancelled
org.apache.spark.SparkException: Job aborted due to stage failure: Stage
7580621 cancelled
18/02/11 13:22:27 WARN TaskSetManager: Lost task 18.0 in stage 7580621.1 (TID
65577140, 10.0.144.170, executor 16): TaskKilled (Stage cancelled)
{code}
According to the above logs, I think the issue is in this line:
https://github.com/apache/spark/blob/1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1281
"Task 18.0 in stage 7580621.0" finished and updated
"shuffleStage.pendingPartitions" when "Task 18.0 in stage 7580621.1" was still
running. Hence, when 18 of 19 tasks finished in "stage 7580621.1", this
condition
(https://github.com/apache/spark/blob/1dc2c1d5e85c5f404f470aeb44c1f3c22786bdea/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L1284)
would be true and trigger "stage 7580621.2".
> java.lang.IllegalStateException: more than one active taskSet for stage
> -----------------------------------------------------------------------
>
> Key: SPARK-23433
> URL: https://issues.apache.org/jira/browse/SPARK-23433
> Project: Spark
> Issue Type: Bug
> Components: Spark Core
> Affects Versions: 2.2.1
> Reporter: Shixiong Zhu
> Priority: Major
>
> This following error thrown by DAGScheduler stopped the cluster:
> {code}
> 18/02/11 13:22:27 ERROR DAGSchedulerEventProcessLoop:
> DAGSchedulerEventProcessLoop failed; shutting down SparkContext
> java.lang.IllegalStateException: more than one active taskSet for stage
> 7580621: 7580621.2,7580621.1
> at
> org.apache.spark.scheduler.TaskSchedulerImpl.submitTasks(TaskSchedulerImpl.scala:229)
> at
> org.apache.spark.scheduler.DAGScheduler.submitMissingTasks(DAGScheduler.scala:1193)
> at
> org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:1059)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:900)
> at
> org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:899)
> at
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
> at
> org.apache.spark.scheduler.DAGScheduler.submitWaitingChildStages(DAGScheduler.scala:899)
> at
> org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:1427)
> at
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1929)
> at
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1880)
> at
> org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1868)
> at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]