-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-19276][CORE] Fetch Failure handling robust to user error handling #16639
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
Changes from 14 commits
b93c37f
9635980
0a60aef
bbef893
730fd83
4494673
14f5125
08491c5
7840480
22da707
2a49705
84eae14
bee5621
ad47611
965506a
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,7 @@ | |
| package org.apache.spark.executor | ||
|
|
||
| import java.io.{File, NotSerializableException} | ||
| import java.lang.Thread.UncaughtExceptionHandler | ||
| import java.lang.management.ManagementFactory | ||
| import java.net.{URI, URL} | ||
| import java.nio.ByteBuffer | ||
|
|
@@ -52,7 +53,8 @@ private[spark] class Executor( | |
| executorHostname: String, | ||
| env: SparkEnv, | ||
| userClassPath: Seq[URL] = Nil, | ||
| isLocal: Boolean = false) | ||
| isLocal: Boolean = false, | ||
| uncaughtExceptionHandler: UncaughtExceptionHandler = SparkUncaughtExceptionHandler) | ||
| extends Logging { | ||
|
|
||
| logInfo(s"Starting executor ID $executorId on host $executorHostname") | ||
|
|
@@ -78,7 +80,7 @@ private[spark] class Executor( | |
| // Setup an uncaught exception handler for non-local mode. | ||
| // Make any thread terminations due to uncaught exceptions kill the entire | ||
| // executor process to avoid surprising stalls. | ||
| Thread.setDefaultUncaughtExceptionHandler(SparkUncaughtExceptionHandler) | ||
| Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler) | ||
| } | ||
|
|
||
| // Start worker thread pool | ||
|
|
@@ -342,6 +344,14 @@ private[spark] class Executor( | |
| } | ||
| } | ||
| } | ||
| task.context.fetchFailed.foreach { fetchFailure => | ||
| // uh-oh. it appears the user code has caught the fetch-failure without throwing any | ||
| // other exceptions. Its *possible* this is what the user meant to do (though highly | ||
| // unlikely). So we will log an error and keep going. | ||
| logError(s"TID ${taskId} completed successfully though internally it encountered " + | ||
| s"unrecoverable fetch failures! Most likely this means user code is incorrectly " + | ||
| s"swallowing Spark's internal ${classOf[FetchFailedException]}", fetchFailure) | ||
| } | ||
| val taskFinish = System.currentTimeMillis() | ||
| val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { | ||
| threadMXBean.getCurrentThreadCpuTime | ||
|
|
@@ -402,8 +412,17 @@ private[spark] class Executor( | |
| execBackend.statusUpdate(taskId, TaskState.FINISHED, serializedResult) | ||
|
|
||
| } catch { | ||
| case ffe: FetchFailedException => | ||
| val reason = ffe.toTaskFailedReason | ||
| case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) => | ||
| val reason = task.context.fetchFailed.get.toTaskFailedReason | ||
| if (!t.isInstanceOf[FetchFailedException]) { | ||
| // there was a fetch failure in the task, but some user code wrapped that exception | ||
| // and threw something else. Regardless, we treat it as a fetch failure. | ||
| val fetchFailedCls = classOf[FetchFailedException].getName | ||
| logWarning(s"TID ${taskId} encountered a ${fetchFailedCls} and " + | ||
| s"failed, but the ${fetchFailedCls} was hidden by another " + | ||
| s"exception. Spark is handling this like a fetch failure and ignoring the " + | ||
| s"other exception: $t") | ||
| } | ||
| setTaskFinishedAndClearInterruptStatus() | ||
| execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: Probably log a similar message as above ?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. do you mean the msg I added about "TID ${taskId} completed successfully though internally it encountered unrecoverable fetch failures!"? I wouldn't think we'd want to log anything special here. I'm trying to make this a "normal" code path. The user is allowed to allowed to do this. (sparksql already does.) we could log a warning, but then this change should be accompanied by auditing the code and making sure we never do this ourselves.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, something along those lines ... |
||
|
|
||
|
|
@@ -455,13 +474,17 @@ private[spark] class Executor( | |
| // Don't forcibly exit unless the exception was inherently fatal, to avoid | ||
| // stopping other tasks unnecessarily. | ||
| if (Utils.isFatalError(t)) { | ||
| SparkUncaughtExceptionHandler.uncaughtException(t) | ||
| uncaughtExceptionHandler.uncaughtException(Thread.currentThread(), t) | ||
| } | ||
|
|
||
| } finally { | ||
| runningTasks.remove(taskId) | ||
| } | ||
| } | ||
|
|
||
| private def hasFetchFailure: Boolean = { | ||
| task != null && task.context != null && task.context.fetchFailed.isDefined | ||
| } | ||
| } | ||
|
|
||
| /** | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,7 +17,7 @@ | |
|
|
||
| package org.apache.spark.shuffle | ||
|
|
||
| import org.apache.spark.{FetchFailed, TaskFailedReason} | ||
| import org.apache.spark.{FetchFailed, TaskContext, TaskFailedReason} | ||
| import org.apache.spark.storage.BlockManagerId | ||
| import org.apache.spark.util.Utils | ||
|
|
||
|
|
@@ -26,6 +26,11 @@ import org.apache.spark.util.Utils | |
| * back to DAGScheduler (through TaskEndReason) so we'd resubmit the previous stage. | ||
| * | ||
| * Note that bmAddress can be null. | ||
| * | ||
| * To prevent user code from hiding this fetch failure, in the constructor we call | ||
| * [[TaskContext.setFetchFailed()]]. This means that you *must* throw this exception immediately | ||
| * after creating it -- you cannot create it, check some condition, and then decide to ignore it | ||
| * (or risk triggering any other exceptions). See SPARK-19276. | ||
| */ | ||
| private[spark] class FetchFailedException( | ||
| bmAddress: BlockManagerId, | ||
|
|
@@ -45,6 +50,12 @@ private[spark] class FetchFailedException( | |
| this(bmAddress, shuffleId, mapId, reduceId, cause.getMessage, cause) | ||
| } | ||
|
|
||
| // SPARK-19276. We set the fetch failure in the task context, so that even if there is user-code | ||
| // which intercepts this exception (possibly wrapping it), the Executor can still tell there was | ||
| // a fetch failure, and send the correct error msg back to the driver. We wrap with an Option | ||
| // because the TaskContext is not defined in some test cases. | ||
| Option(TaskContext.get()).map(_.setFetchFailed(this)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since creation of an Exception does not necessarily mean it should get thrown - we must explicitly add this expectation to the documentation/contract of FetchFailedException constructor - indicating that we expect it to be created only for it to be thrown immediately.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, good point. I added to the docs, does it look OK? I also considered making the call to |
||
|
|
||
| def toTaskFailedReason: TaskFailedReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, | ||
| Utils.exceptionString(this)) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,6 +18,7 @@ | |
| package org.apache.spark.executor | ||
|
|
||
| import java.io.{Externalizable, ObjectInput, ObjectOutput} | ||
| import java.lang.Thread.UncaughtExceptionHandler | ||
| import java.nio.ByteBuffer | ||
| import java.util.Properties | ||
| import java.util.concurrent.{CountDownLatch, TimeUnit} | ||
|
|
@@ -27,7 +28,7 @@ import scala.concurrent.duration._ | |
|
|
||
| import org.mockito.ArgumentCaptor | ||
| import org.mockito.Matchers.{any, eq => meq} | ||
| import org.mockito.Mockito.{inOrder, when} | ||
| import org.mockito.Mockito.{inOrder, verify, when} | ||
| import org.mockito.invocation.InvocationOnMock | ||
| import org.mockito.stubbing.Answer | ||
| import org.scalatest.concurrent.Eventually | ||
|
|
@@ -37,9 +38,12 @@ import org.apache.spark._ | |
| import org.apache.spark.TaskState.TaskState | ||
| import org.apache.spark.memory.MemoryManager | ||
| import org.apache.spark.metrics.MetricsSystem | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.rpc.RpcEnv | ||
| import org.apache.spark.scheduler.{FakeTask, TaskDescription} | ||
| import org.apache.spark.scheduler.{FakeTask, ResultTask, TaskDescription} | ||
| import org.apache.spark.serializer.JavaSerializer | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
| import org.apache.spark.storage.BlockManagerId | ||
|
|
||
| class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSugar with Eventually { | ||
|
|
||
|
|
@@ -123,6 +127,75 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug | |
| } | ||
| } | ||
|
|
||
| test("SPARK-19276: Handle FetchFailedExceptions that are hidden by user exceptions") { | ||
| val conf = new SparkConf().setMaster("local").setAppName("executor suite test") | ||
| sc = new SparkContext(conf) | ||
| val serializer = SparkEnv.get.closureSerializer.newInstance() | ||
| val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size | ||
|
|
||
| // Submit a job where a fetch failure is thrown, but user code has a try/catch which hides | ||
| // the fetch failure. The executor should still tell the driver that the task failed due to a | ||
| // fetch failure, not a generic exception from user code. | ||
| val inputRDD = new FetchFailureThrowingRDD(sc) | ||
| val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = false) | ||
| val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) | ||
| val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() | ||
| val task = new ResultTask( | ||
| stageId = 1, | ||
| stageAttemptId = 0, | ||
| taskBinary = taskBinary, | ||
| partition = secondRDD.partitions(0), | ||
| locs = Seq(), | ||
| outputId = 0, | ||
| localProperties = new Properties(), | ||
| serializedTaskMetrics = serializedTaskMetrics | ||
| ) | ||
|
|
||
| val serTask = serializer.serialize(task) | ||
| val taskDescription = createFakeTaskDescription(serTask) | ||
|
|
||
| val failReason = runTaskAndGetFailReason(taskDescription) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can you add a comment about what's going on here? I think the FFE gets thrown because the shuffle map data was never generated? And then you're checking that it's correctly accounted for, even though the user RDD code wrapped the exception in something else? |
||
| assert(failReason.isInstanceOf[FetchFailed]) | ||
| } | ||
|
|
||
| test("SPARK-19276: OOMs correctly handled with a FetchFailure") { | ||
| // when there is a fatal error like an OOM, we don't do normal fetch failure handling, since it | ||
| // may be a false positive. And we should call the uncaught exception handler. | ||
| val conf = new SparkConf().setMaster("local").setAppName("executor suite test") | ||
| sc = new SparkContext(conf) | ||
| val serializer = SparkEnv.get.closureSerializer.newInstance() | ||
| val resultFunc = (context: TaskContext, itr: Iterator[Int]) => itr.size | ||
|
|
||
| // Submit a job where a fetch failure is thrown, but user code has a try/catch which hides | ||
| // the fetch failure. The executor should still tell the driver that the task failed due to a | ||
| // fetch failure, not a generic exception from user code. | ||
| val inputRDD = new FetchFailureThrowingRDD(sc) | ||
| val secondRDD = new FetchFailureHidingRDD(sc, inputRDD, throwOOM = true) | ||
| val taskBinary = sc.broadcast(serializer.serialize((secondRDD, resultFunc)).array()) | ||
| val serializedTaskMetrics = serializer.serialize(TaskMetrics.registered).array() | ||
| val task = new ResultTask( | ||
| stageId = 1, | ||
| stageAttemptId = 0, | ||
| taskBinary = taskBinary, | ||
| partition = secondRDD.partitions(0), | ||
| locs = Seq(), | ||
| outputId = 0, | ||
| localProperties = new Properties(), | ||
| serializedTaskMetrics = serializedTaskMetrics | ||
| ) | ||
|
|
||
| val serTask = serializer.serialize(task) | ||
| val taskDescription = createFakeTaskDescription(serTask) | ||
|
|
||
| val (failReason, uncaughtExceptionHandler) = | ||
| runTaskGetFailReasonAndExceptionHandler(taskDescription) | ||
| assert(failReason.isInstanceOf[ExceptionFailure]) | ||
| val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable]) | ||
| verify(uncaughtExceptionHandler).uncaughtException(any(), exceptionCaptor.capture()) | ||
| assert(exceptionCaptor.getAllValues.size === 1) | ||
| assert(exceptionCaptor.getAllValues.get(0).isInstanceOf[OutOfMemoryError]) | ||
| } | ||
|
|
||
| test("Gracefully handle error in task deserialization") { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. is this test related to this PR? (seems useful but like it should be in its own PR?)
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @mridulm pointed out this bug in an earlier version of this pr, so I fixed the bug and added a test case. But in any case, I've separated this out into #16930 / https://issues.apache.org/jira/browse/SPARK-19597 |
||
| val conf = new SparkConf | ||
| val serializer = new JavaSerializer(conf) | ||
|
|
@@ -169,13 +242,20 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug | |
| } | ||
|
|
||
| private def runTaskAndGetFailReason(taskDescription: TaskDescription): TaskFailedReason = { | ||
| runTaskGetFailReasonAndExceptionHandler(taskDescription)._1 | ||
| } | ||
|
|
||
| private def runTaskGetFailReasonAndExceptionHandler( | ||
| taskDescription: TaskDescription): (TaskFailedReason, UncaughtExceptionHandler) = { | ||
| val mockBackend = mock[ExecutorBackend] | ||
| val mockUncaughtExceptionHandler = mock[UncaughtExceptionHandler] | ||
| var executor: Executor = null | ||
| try { | ||
| executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true) | ||
| executor = new Executor("id", "localhost", SparkEnv.get, userClassPath = Nil, isLocal = true, | ||
| uncaughtExceptionHandler = mockUncaughtExceptionHandler) | ||
| // the task will be launched in a dedicated worker thread | ||
| executor.launchTask(mockBackend, taskDescription) | ||
| eventually(timeout(5 seconds), interval(10 milliseconds)) { | ||
| eventually(timeout(5.seconds), interval(10.milliseconds)) { | ||
| assert(executor.numRunningTasks === 0) | ||
| } | ||
| } finally { | ||
|
|
@@ -193,7 +273,56 @@ class ExecutorSuite extends SparkFunSuite with LocalSparkContext with MockitoSug | |
| assert(statusCaptor.getAllValues().get(0).remaining() === 0) | ||
| // second update is more interesting | ||
| val failureData = statusCaptor.getAllValues.get(1) | ||
| SparkEnv.get.closureSerializer.newInstance().deserialize[TaskFailedReason](failureData) | ||
| val failReason = | ||
| SparkEnv.get.closureSerializer.newInstance().deserialize[TaskFailedReason](failureData) | ||
| (failReason, mockUncaughtExceptionHandler) | ||
| } | ||
| } | ||
|
|
||
| class FetchFailureThrowingRDD(sc: SparkContext) extends RDD[Int](sc, Nil) { | ||
| override def compute(split: Partition, context: TaskContext): Iterator[Int] = { | ||
| new Iterator[Int] { | ||
| override def hasNext: Boolean = true | ||
| override def next(): Int = { | ||
| throw new FetchFailedException( | ||
| bmAddress = BlockManagerId("1", "hostA", 1234), | ||
| shuffleId = 0, | ||
| mapId = 0, | ||
| reduceId = 0, | ||
| message = "fake fetch failure" | ||
| ) | ||
| } | ||
| } | ||
| } | ||
| override protected def getPartitions: Array[Partition] = { | ||
| Array(new SimplePartition) | ||
| } | ||
| } | ||
|
|
||
| class SimplePartition extends Partition { | ||
| override def index: Int = 0 | ||
| } | ||
|
|
||
| class FetchFailureHidingRDD( | ||
| sc: SparkContext, | ||
| val input: FetchFailureThrowingRDD, | ||
| throwOOM: Boolean) extends RDD[Int](input) { | ||
| override def compute(split: Partition, context: TaskContext): Iterator[Int] = { | ||
| val inItr = input.compute(split, context) | ||
| try { | ||
| Iterator(inItr.size) | ||
| } catch { | ||
| case t: Throwable => | ||
| if (throwOOM) { | ||
| throw new OutOfMemoryError("OOM while handling another exception") | ||
| } else { | ||
| throw new RuntimeException("User Exception that hides the original exception", t) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| override protected def getPartitions: Array[Partition] = { | ||
| Array(new SimplePartition) | ||
| } | ||
| } | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
tiny nit: but does it make sense to store the taskFailedReason (rather than the actual exception) in the task context?