[jira] [Commented] (SPARK-14658) when executor lost DagScheduer may submit one stage twice even if the first running taskset for this stage is not finished
[ https://issues.apache.org/jira/browse/SPARK-14658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15870816#comment-15870816 ] Josh Rosen commented on SPARK-14658: Here's the logs from my reproduction, excerpted down to only the relevant parts (as near as I can tell): First attempt of task set being submitted: {code} 17/02/13 20:11:59 INFO DAGScheduler: waiting: Set(ShuffleMapStage 3086, ResultStage 3087) 17/02/13 20:11:59 INFO DAGScheduler: failed: Set() 17/02/13 20:11:59 INFO DAGScheduler: Submitting ShuffleMapStage 3086 (MapPartitionsRDD[34696] at cache at :61), which has no missing parents 17/02/13 20:11:59 INFO MemoryStore: Block broadcast_2871 stored as values in memory (estimated size 67.1 KB, free 9.1 GB) 17/02/13 20:11:59 INFO MemoryStore: Block broadcast_2871_piece0 stored as bytes in memory (estimated size 28.1 KB, free 9.1 GB) 17/02/13 20:11:59 INFO BlockManagerInfo: Added broadcast_2871_piece0 in memory on :45333 (size: 28.1 KB, free: 10.6 GB) 17/02/13 20:11:59 INFO SparkContext: Created broadcast 2871 from broadcast at DAGScheduler.scala:996 17/02/13 20:11:59 INFO DAGScheduler: Submitting 2213 missing tasks from ShuffleMapStage 3086 (MapPartitionsRDD[34696] at cache at :61) 17/02/13 20:11:59 INFO TaskSchedulerImpl: Adding task set 3086.0 with 2213 tasks 17/02/13 20:11:59 INFO FairSchedulableBuilder: Added task set TaskSet_3086.0 tasks to pool 1969095006217179029 {code} While the task set was running some tasks failed due to fetch failures from the parent stage, causing both the stage with the fetch failures and the parent stage to be resubmitted: {code} 17/02/13 20:44:34 WARN TaskSetManager: Lost task 1213.0 in stage 3086.0 (TID 370751, , executor 622): FetchFailed(null, shuffleId=638, mapId=-1, reduceId=0, message= org.apache.spark.shuffle.MetadataFetchFailedException: Missing an output location for shuffle 638 at org.apache.spark.MapOutputTracker$$anonfun$org$apache$spark$MapOutputTracker$$convertMapStatuses$2.apply(MapOutputTracker.scala:697) at org.apache.spark.MapOutputTracker$$anonfun$org$apache$spark$MapOutputTracker$$convertMapStatuses$2.apply(MapOutputTracker.scala:693) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772) at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33) at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771) at org.apache.spark.MapOutputTracker$.org$apache$spark$MapOutputTracker$$convertMapStatuses(MapOutputTracker.scala:693) at org.apache.spark.MapOutputTracker.getMapSizesByExecutorId(MapOutputTracker.scala:147) at org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:49) at org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:169) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:336) at org.apache.spark.rdd.RDD$$anonfun$8.apply(RDD.scala:334) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:957) at org.apache.spark.storage.BlockManager$$anonfun$doPutIterator$1.apply(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:888) at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948) at org.apache.spark.storage.BlockManager.getOrElseUpdate(BlockManager.scala:694) at org.apache.spark.rdd.RDD.getOrCompute(RDD.scala:334) at org.apache.spark.rdd.RDD.iterator(RDD.scala:285) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:105) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323) at org.apache.spark.rdd.RDD.iterator(RDD.scala:287) at
[jira] [Commented] (SPARK-14658) when executor lost DagScheduer may submit one stage twice even if the first running taskset for this stage is not finished
[ https://issues.apache.org/jira/browse/SPARK-14658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15870777#comment-15870777 ] Josh Rosen commented on SPARK-14658: [~srowen], I think that [~yixiaohua] is right here: it looks like SPARK-14649 is proposing a performance optimization to avoid the wasteful resubmission of tasks when failures occur (which is only a perf. optimization, although an important one for the workload that [~sitalke...@gmail.com] describes), whereas this ticket is discussing a correctness issue where one of the scheduler's internal invariants is being violated, causing a total SparkContext shutdown. I'm going to unmark this as a duplicate and will re-open so that someone can review the proposed fix. I also have additional logs from a fresh occurrence of this bug in Spark 2.1.0+, which I'll upload in a followup comment here. > when executor lost DagScheduer may submit one stage twice even if the first > running taskset for this stage is not finished > -- > > Key: SPARK-14658 > URL: https://issues.apache.org/jira/browse/SPARK-14658 > Project: Spark > Issue Type: Bug > Components: Scheduler >Affects Versions: 1.6.1, 2.0.0, 2.1.0, 2.2.0 > Environment: spark1.6.1 hadoop-2.6.0-cdh5.4.2 >Reporter: yixiaohua > > 16/04/14 15:35:22 ERROR DAGSchedulerEventProcessLoop: > DAGSchedulerEventProcessLoop failed; shutting down SparkContext > java.lang.IllegalStateException: more than one active taskSet for stage 57: > 57.2,57.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) > First Time: > 16/04/14 15:35:20 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 5, 8, 9, 12, > 13, 16, 17, 18, 19, 23, 26, 27, 28, 29, 30, 31, 40, 42, 43, 48, 49, 50, 51, > 52, 53, 55, 56, 57, 59, 60, 61, 67, 70, 71, 84, 85, 86, 87, 98, 99, 100, 101, > 108, 109, 110, 111, 112, 113, 114, 115, 126, 127, 134, 136, 137, 146, 147, > 150, 151, 154, 155, 158, 159, 162, 163, 164, 165, 166, 167, 170, 171, 172, > 173, 174, 175, 176, 177, 178, 179, 180, 181, 188, 189, 190, 191, 198, 199, > 204, 206, 207, 208, 218, 219, 222, 223, 230, 231, 236, 238, 239 > 16/04/14 15:35:20 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:20 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:20 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:20 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:20 INFO DAGScheduler: Submitting 100 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:20 DEBUG DAGScheduler: New pending partitions: Set(206, 177, > 127, 98, 48, 27, 23, 163, 238, 188, 159, 28, 109, 59, 9, 176, 126, 207, 174, > 43, 170, 208, 158, 108, 29, 8, 204, 154, 223, 173, 219, 190, 111, 61, 40, > 136, 115, 86, 57, 155, 55, 230, 222, 180, 172, 151, 101, 18, 166, 56, 137, > 87, 52, 171, 71, 42, 167, 198, 67, 17, 236, 165, 13, 5, 53, 178, 99, 70, 49, > 218, 147, 164, 114, 85, 60, 31, 179, 150, 19, 100, 50, 175, 146, 134, 113, > 84, 51, 30, 199, 26, 16, 191, 162, 112, 12, 239, 231, 189, 181, 110) > Second Time: > 16/04/14 15:35:22 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 26 > 16/04/14 15:35:22 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:22 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:22 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:22 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:22 INFO DAGScheduler: Submitting 1 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:22
[jira] [Commented] (SPARK-14658) when executor lost DagScheduer may submit one stage twice even if the first running taskset for this stage is not finished
[ https://issues.apache.org/jira/browse/SPARK-14658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15253520#comment-15253520 ] yixiaohua commented on SPARK-14658: --- Owen thanks for your attention ,but i think it is not the same,the 14649 jira is running duplicate tasks, but this jira is submit a task set for stage even if there are another active tast set is active which can cause the spark context exit. > when executor lost DagScheduer may submit one stage twice even if the first > running taskset for this stage is not finished > -- > > Key: SPARK-14658 > URL: https://issues.apache.org/jira/browse/SPARK-14658 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.6.1 > Environment: spark1.6.1 hadoop-2.6.0-cdh5.4.2 >Reporter: yixiaohua > > 16/04/14 15:35:22 ERROR DAGSchedulerEventProcessLoop: > DAGSchedulerEventProcessLoop failed; shutting down SparkContext > java.lang.IllegalStateException: more than one active taskSet for stage 57: > 57.2,57.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) > First Time: > 16/04/14 15:35:20 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 5, 8, 9, 12, > 13, 16, 17, 18, 19, 23, 26, 27, 28, 29, 30, 31, 40, 42, 43, 48, 49, 50, 51, > 52, 53, 55, 56, 57, 59, 60, 61, 67, 70, 71, 84, 85, 86, 87, 98, 99, 100, 101, > 108, 109, 110, 111, 112, 113, 114, 115, 126, 127, 134, 136, 137, 146, 147, > 150, 151, 154, 155, 158, 159, 162, 163, 164, 165, 166, 167, 170, 171, 172, > 173, 174, 175, 176, 177, 178, 179, 180, 181, 188, 189, 190, 191, 198, 199, > 204, 206, 207, 208, 218, 219, 222, 223, 230, 231, 236, 238, 239 > 16/04/14 15:35:20 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:20 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:20 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:20 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:20 INFO DAGScheduler: Submitting 100 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:20 DEBUG DAGScheduler: New pending partitions: Set(206, 177, > 127, 98, 48, 27, 23, 163, 238, 188, 159, 28, 109, 59, 9, 176, 126, 207, 174, > 43, 170, 208, 158, 108, 29, 8, 204, 154, 223, 173, 219, 190, 111, 61, 40, > 136, 115, 86, 57, 155, 55, 230, 222, 180, 172, 151, 101, 18, 166, 56, 137, > 87, 52, 171, 71, 42, 167, 198, 67, 17, 236, 165, 13, 5, 53, 178, 99, 70, 49, > 218, 147, 164, 114, 85, 60, 31, 179, 150, 19, 100, 50, 175, 146, 134, 113, > 84, 51, 30, 199, 26, 16, 191, 162, 112, 12, 239, 231, 189, 181, 110) > Second Time: > 16/04/14 15:35:22 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 26 > 16/04/14 15:35:22 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:22 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:22 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:22 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:22 INFO DAGScheduler: Submitting 1 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:22 DEBUG DAGScheduler: New pending partitions: Set(26) -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org
[jira] [Commented] (SPARK-14658) when executor lost DagScheduer may submit one stage twice even if the first running taskset for this stage is not finished
[ https://issues.apache.org/jira/browse/SPARK-14658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15249527#comment-15249527 ] yixiaohua commented on SPARK-14658: --- https://github.com/apache/spark/pull/12524 > when executor lost DagScheduer may submit one stage twice even if the first > running taskset for this stage is not finished > -- > > Key: SPARK-14658 > URL: https://issues.apache.org/jira/browse/SPARK-14658 > Project: Spark > Issue Type: Bug > Components: Spark Core >Affects Versions: 1.6.1 > Environment: spark1.6.1 hadoop-2.6.0-cdh5.4.2 >Reporter: yixiaohua > > 16/04/14 15:35:22 ERROR DAGSchedulerEventProcessLoop: > DAGSchedulerEventProcessLoop failed; shutting down SparkContext > java.lang.IllegalStateException: more than one active taskSet for stage 57: > 57.2,57.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) > First Time: > 16/04/14 15:35:20 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 5, 8, 9, 12, > 13, 16, 17, 18, 19, 23, 26, 27, 28, 29, 30, 31, 40, 42, 43, 48, 49, 50, 51, > 52, 53, 55, 56, 57, 59, 60, 61, 67, 70, 71, 84, 85, 86, 87, 98, 99, 100, 101, > 108, 109, 110, 111, 112, 113, 114, 115, 126, 127, 134, 136, 137, 146, 147, > 150, 151, 154, 155, 158, 159, 162, 163, 164, 165, 166, 167, 170, 171, 172, > 173, 174, 175, 176, 177, 178, 179, 180, 181, 188, 189, 190, 191, 198, 199, > 204, 206, 207, 208, 218, 219, 222, 223, 230, 231, 236, 238, 239 > 16/04/14 15:35:20 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:20 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:20 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:20 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:20 INFO DAGScheduler: Submitting 100 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:20 DEBUG DAGScheduler: New pending partitions: Set(206, 177, > 127, 98, 48, 27, 23, 163, 238, 188, 159, 28, 109, 59, 9, 176, 126, 207, 174, > 43, 170, 208, 158, 108, 29, 8, 204, 154, 223, 173, 219, 190, 111, 61, 40, > 136, 115, 86, 57, 155, 55, 230, 222, 180, 172, 151, 101, 18, 166, 56, 137, > 87, 52, 171, 71, 42, 167, 198, 67, 17, 236, 165, 13, 5, 53, 178, 99, 70, 49, > 218, 147, 164, 114, 85, 60, 31, 179, 150, 19, 100, 50, 175, 146, 134, 113, > 84, 51, 30, 199, 26, 16, 191, 162, 112, 12, 239, 231, 189, 181, 110) > Second Time: > 16/04/14 15:35:22 INFO DAGScheduler: Resubmitting ShuffleMapStage 57 (run at > AccessController.java:-2) because some of its tasks had failed: 26 > 16/04/14 15:35:22 DEBUG DAGScheduler: submitStage(ShuffleMapStage 57) > 16/04/14 15:35:22 DEBUG DAGScheduler: missing: List() > 16/04/14 15:35:22 INFO DAGScheduler: Submitting ShuffleMapStage 57 > (MapPartitionsRDD[7887] at run at AccessController.java:-2), which has no > missing parents > 16/04/14 15:35:22 DEBUG DAGScheduler: submitMissingTasks(ShuffleMapStage 57) > 16/04/14 15:35:22 INFO DAGScheduler: Submitting 1 missing tasks from > ShuffleMapStage 57 (MapPartitionsRDD[7887] at run at AccessController.java:-2) > 16/04/14 15:35:22 DEBUG DAGScheduler: New pending partitions: Set(26) -- This message was sent by Atlassian JIRA (v6.3.4#6332) - To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org