liupengcheng created SPARK-24897:
------------------------------------
Summary: DAGScheduler should not unregisterMapOutput and
increaseEpoch repeatedly for stage fetchFailed
Key: SPARK-24897
URL: https://issues.apache.org/jira/browse/SPARK-24897
Project: Spark
Issue Type: Bug
Components: Scheduler, Spark Core
Affects Versions: 2.3.1, 2.1.0
Reporter: liupengcheng
In Spark2.1, when a stage fetchfailed DAGScheduler will retry both this stage
and it's parent stage, however, when the parent stage is resubmitted and start
running, the mapstatuses can
still be invalidate by the stage's outstanding task due to fetchfailed.
The stage's outstanding task might unregister the mapstatuses with new epoch,
thus causing
the parent stage repeated MetadataFetchFailed and finally failling the Job.
{code:java}
2018-07-23,01:52:33,012 WARN org.apache.spark.scheduler.TaskSetManager: Lost
task 174.0 in stage 71.0 (TID 154127, <host>, executor 96):
FetchFailed(BlockManagerId(4945, <host>, 22409), shuffleId=24, mapId=667,
reduceId=174, message= org.apache.spark.shuffle.FetchFailedException: Failed to
connect to <host>/<ip>:22409
2018-07-23,01:52:33,013 INFO org.apache.spark.scheduler.DAGScheduler: Marking
ShuffleMapStage 71 (map at DeviceLocateMain.scala:238) as failed due to a fetch
failure from ShuffleMapStage 69 ($plus$plus at DeviceLocateMain.scala:236)
2018-07-23,01:52:33,014 INFO org.apache.spark.scheduler.DAGScheduler:
ShuffleMapStage 71 (map at DeviceLocateMain.scala:238) failed in 246.856 s
2018-07-23,01:52:33,014 INFO org.apache.spark.scheduler.DAGScheduler:
Resubmitting ShuffleMapStage 69 ($plus$plus at DeviceLocateMain.scala:236) and
ShuffleMapStage 71 (map at DeviceLocateMain.scala:238) due to fetch failure
2018-07-23,01:52:36,004 WARN org.apache.spark.scheduler.TaskSetManager: Lost
task 120.0 in stage 71.0 (TID 154073, <host>, executor 286):
FetchFailed(BlockManagerId(4208, <host>, 22409), shuffleId=24, mapId=241,
reduceId=120, message= org.apache.spark.shuffle.FetchFailedException: Failed to
connect to <host>/<ip>:22409
2018-07-23,01:52:36,005 INFO org.apache.spark.scheduler.DAGScheduler:
Resubmitting ShuffleMapStage 69 ($plus$plus at DeviceLocateMain.scala:236) and
ShuffleMapStage 71 (map at DeviceLocateMain.scala:238) due to fetch failure
2018-07-23,01:52:36,017 INFO org.apache.spark.storage.MemoryStore: Block
broadcast_63_piece0 stored as bytes in memory (estimated size 4.0 MB, free 26.7
MB)
2018-07-23,01:52:36,025 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_59_piece1 on <host>:52349 in memory (size: 4.0 MB, free: 3.0 GB)
2018-07-23,01:52:36,029 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_61_piece6 on <host>:52349 in memory (size: 4.0 MB, free: 3.0 GB)
2018-07-23,01:52:36,079 INFO org.apache.spark.deploy.yarn.YarnAllocator:
Canceling requests for 0 executor containers
2018-07-23,01:52:36,079 WARN org.apache.spark.deploy.yarn.YarnAllocator:
Expected to find pending requests, but found none.
2018-07-23,01:52:36,094 INFO org.apache.spark.storage.BlockManagerInfo: Added
broadcast_63_piece0 in memory on <host>:56780 (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,095 INFO org.apache.spark.storage.MemoryStore: Block
broadcast_63_piece1 stored as bytes in memory (estimated size 4.0 MB, free 30.7
MB)
2018-07-23,01:52:36,107 INFO org.apache.spark.storage.BlockManagerInfo: Added
broadcast_63_piece1 in memory on <host>:56780 (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,108 INFO org.apache.spark.storage.MemoryStore: Block
broadcast_63_piece2 stored as bytes in memory (estimated size 4.0 MB, free 34.7
MB)
2018-07-23,01:52:36,108 INFO org.apache.spark.storage.BlockManagerInfo: Added
broadcast_63_piece2 in memory on <host>:56780 (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,108 INFO org.apache.spark.storage.MemoryStore: Block
broadcast_63_piece3 stored as bytes in memory (estimated size 4.0 MB, free 38.7
MB)
2018-07-23,01:52:36,132 INFO org.apache.spark.storage.BlockManagerInfo: Added
broadcast_63_piece3 in memory on <host>:56780 (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,132 INFO org.apache.spark.storage.MemoryStore: Block
broadcast_63_piece4 stored as bytes in memory (estimated size 3.8 MB, free 42.5
MB)
2018-07-23,01:52:36,132 INFO org.apache.spark.storage.BlockManagerInfo: Added
broadcast_63_piece4 in memory on <host>:56780 (size: 3.8 MB, free: 3.7 GB)
2018-07-23,01:52:36,132 INFO org.apache.spark.MapOutputTracker: Broadcast
mapstatuses size = 384, actual size = 20784475
2018-07-23,01:52:36,132 INFO org.apache.spark.MapOutputTrackerMaster: Size of
output statuses for shuffle 17 is 384 bytes
2018-07-23,01:52:36,133 INFO org.apache.spark.MapOutputTrackerMaster: Epoch
changed, not caching!
2018-07-23,01:52:36,185 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_61_piece3 on <host>:52349 in memory (size: 4.0 MB, free: 3.0 GB)
2018-07-23,01:52:36,185 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_63_piece4 on <host>:56780 in memory (size: 3.8 MB, free: 3.7 GB)
2018-07-23,01:52:36,186 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_63_piece2 on <host>:56780 in memory (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,186 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_63_piece1 on <host>:56780 in memory (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,186 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_63_piece0 on <host>:56780 in memory (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,186 INFO org.apache.spark.storage.BlockManagerInfo: Removed
broadcast_63_piece3 on <host>:56780 in memory (size: 4.0 MB, free: 3.7 GB)
2018-07-23,01:52:36,192 WARN org.apache.spark.scheduler.TaskSetManager: Lost
task 1.0 in stage 69.1 (TID 154955, <host>, executor 267): FetchFailed(null,
shuffleId=17, mapId=-1, reduceId=-1, message=
org.apache.spark.shuffle.MetadataFetchFailedException: java.io.IOException:
org.apache.spark.SparkException: Failed to get broadcast_63_piece4 of
broadcast_63
{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]