Skip to content

[SPARK-4085] Propagate FetchFailedException when Spark fails to read local shuffle file. #3579

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.spark.storage

import java.io.{InputStream, IOException}
import java.util.concurrent.LinkedBlockingQueue

import scala.collection.mutable.{ArrayBuffer, HashSet, Queue}
Expand Down Expand Up @@ -289,17 +290,22 @@ final class ShuffleBlockFetcherIterator(
}

val iteratorTry: Try[Iterator[Any]] = result match {
case FailureFetchResult(_, e) => Failure(e)
case SuccessFetchResult(blockId, _, buf) => {
val is = blockManager.wrapForCompression(blockId, buf.createInputStream())
val iter = serializer.newInstance().deserializeStream(is).asIterator
Success(CompletionIterator[Any, Iterator[Any]](iter, {
// Once the iterator is exhausted, release the buffer and set currentResult to null
// so we don't release it again in cleanup.
currentResult = null
buf.release()
}))
}
case FailureFetchResult(_, e) =>
Failure(e)
case SuccessFetchResult(blockId, _, buf) =>
// There is a chance that createInputStream can fail (e.g. fetching a local file that does
// not exist, SPARK-4085). In that case, we should propagate the right exception so
// the scheduler gets a FetchFailedException.
Try(buf.createInputStream()).map { is0 =>
val is = blockManager.wrapForCompression(blockId, is0)
val iter = serializer.newInstance().deserializeStream(is).asIterator
CompletionIterator[Any, Iterator[Any]](iter, {
// Once the iterator is exhausted, release the buffer and set currentResult to null
// so we don't release it again in cleanup.
currentResult = null
buf.release()
})
}
}

(result.blockId, iteratorTry)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,8 +17,6 @@

package org.apache.spark

import java.util.concurrent.atomic.AtomicInteger

import org.scalatest.BeforeAndAfterAll

import org.apache.spark.network.TransportContext
Expand Down
23 changes: 23 additions & 0 deletions core/src/test/scala/org/apache/spark/ShuffleSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import org.scalatest.Matchers
import org.apache.spark.ShuffleSuite.NonJavaSerializableClass
import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD}
import org.apache.spark.serializer.KryoSerializer
import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId}
import org.apache.spark.util.MutablePair

abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext {
Expand Down Expand Up @@ -263,6 +264,28 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex
}
}
}

test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") {
val myConf = conf.clone().set("spark.test.noStageRetry", "false")
sc = new SparkContext("local", "test", myConf)
val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _)
rdd.count()

// Delete one of the local shuffle blocks.
val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0))
val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0))
assert(hashFile.exists() || sortFile.exists())

if (hashFile.exists()) {
hashFile.delete()
}
if (sortFile.exists()) {
sortFile.delete()
}

// This count should retry the execution of the previous stage and rerun shuffle.
rdd.count()
}
}

object ShuffleSuite {
Expand Down