Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-26612

Speculation kill causing finished stage recomputed

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.1.0
    • 2.2.2, 2.3.1, 2.4.0
    • Scheduler, Spark Core
    • None

    Description

      In our production spark cluster, we encoutered this issue.

      A more detailed explaination:

      Let's say we have two stage: stage0.0 and stage1.0, and stage 0 is a shuffleMapStage, and stage1 has dependency on stage0, and we enabled spark.speculation.

      when task0.0 of stage1.0 finished, and is trying to kill task0.1(speculative) of stage1.0, task0.1 throws a wrapped FetchFailedException whose root cause is  java.nio.channels.ClosedByInterruptException(caused by speculation kill).

      Exception stack:

      at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator$$anonfun$hasNext$1.apply$mcZ$sp(BlockStoreShuffleReader.scala:148)
      at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator$$anonfun$hasNext$1.apply(BlockStoreShuffleReader.scala:148)
      at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator$$anonfun$hasNext$1.apply(BlockStoreShuffleReader.scala:148)
      at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator.tryThrowFetchFailedException(BlockStoreShuffleReader.scala:127)
      at org.apache.spark.shuffle.BlockStoreShuffleReader$ProcessFetchFailedIterator.hasNext(BlockStoreShuffleReader.scala:148)
      at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
      at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:32)
      at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
      at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:438)
      at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
      at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
      at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:439)
      at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:439)
      at scala.collection.Iterator$$anon$11.hasNext(Iterator.scala:408)
      at org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)
      at org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)
      at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)
      at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
      at org.apache.spark.scheduler.Task.run(Task.scala:99)
      at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:308)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
      at java.lang.Thread.run(Thread.java:745)
      Caused by: java.io.IOException: Error in opening FileSegmentManagedBuffer{file=/home/work/hdd6/yarn/c3prc-hadoop/nodemanager/usercache/h_user_profile/appcache/application_1505730831071_76097/blockmgr-bb226ff8-dd5f-4296-b3cc-ce7ff5cc60cc/37/shuffle_1_1182_0.data, offset=17789166, length=35709}
      at org.apache.spark.network.buffer.FileSegmentManagedBuffer.createInputStream(FileSegmentManagedBuffer.java:114)
      at org.apache.spark.storage.ShuffleBlockFetcherIterator.next(ShuffleBlockFetcherIterator.scala:371)
      ... 26 more
      Caused by: java.nio.channels.ClosedByInterruptException
      at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202)
      at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:155)
      at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:65)
      at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:109)
      

      Seems in latest spark version, this problem still exists! FetchFailedException might be throwed in ShuffleBlockFetcherIterator.next, where the task is accessing local shuffle block or encountering a stream corruption. 

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              liupengcheng liupengcheng
              Votes:
              0 Vote for this issue
              Watchers:
              1 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: