[ https://issues.apache.org/jira/browse/SPARK-20178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15969388#comment-15969388 ]
Thomas Graves commented on SPARK-20178: --------------------------------------- One thing I ran into today which is somewhat related to this is a combination of failure types. In this case it was broadcast fetch failures combined with shuffle fetch failures which lead to 4 task failures and failed the job. I believe they were all from the same host and happened really quickly (within 3 seconds). This seems like this should fall under the fetch failure case as well. Failed to get broadcast_646_piece0 of broadcast_646 at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1222) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:165) at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:64) at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:64) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:88) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70) at org.apache.spark.MapOutputTracker$.deserializeMapStatuses(MapOutputTracker.scala:691) at org.apache.spark.MapOutputTracker.getStatuses(MapOutputTracker.scala:204) at org.apache.spark.MapOutputTracker.getMapSizesByExecutorId(MapOutputTracker.scala:143) at org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:47) at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$2.apply(CoGroupedRDD.scala:147) at org.apache.spark.rdd.CoGroupedRDD$$anonfun$compute$2.apply(CoGroupedRDD.scala:136) at scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772) at scala.collection.immutable.List.foreach(List.scala:318) at scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771) at org.apache.spark.rdd.CoGroupedRDD.compute(CoGroupedRDD.scala:136) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306) > Improve Scheduler fetch failures > -------------------------------- > > Key: SPARK-20178 > URL: https://issues.apache.org/jira/browse/SPARK-20178 > Project: Spark > Issue Type: Epic > Components: Scheduler > Affects Versions: 2.1.0 > Reporter: Thomas Graves > > We have been having a lot of discussions around improving the handling of > fetch failures. There are 4 jira currently related to this. > We should try to get a list of things we want to improve and come up with one > cohesive design. > SPARK-20163, SPARK-20091, SPARK-14649 , and SPARK-19753 > I will put my initial thoughts in a follow on comment. -- 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