Skip to content

Commit a3bca65

Browse files
committed
Send the fetch failure message back to Web UI
1 parent 0734d09 commit a3bca65

File tree

12 files changed

+89
-57
lines changed

12 files changed

+89
-57
lines changed

core/src/main/scala/org/apache/spark/TaskEndReason.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -69,11 +69,13 @@ case class FetchFailed(
6969
bmAddress: BlockManagerId, // Note that bmAddress can be null
7070
shuffleId: Int,
7171
mapId: Int,
72-
reduceId: Int)
72+
reduceId: Int,
73+
message: String)
7374
extends TaskFailedReason {
7475
override def toErrorString: String = {
7576
val bmAddressString = if (bmAddress == null) "null" else bmAddress.toString
76-
s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId)"
77+
s"FetchFailed($bmAddressString, shuffleId=$shuffleId, mapId=$mapId, reduceId=$reduceId, " +
78+
s"message=\n$message\n)"
7779
}
7880
}
7981

core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1050,7 +1050,7 @@ class DAGScheduler(
10501050
logInfo("Resubmitted " + task + ", so marking it as still running")
10511051
stage.pendingTasks += task
10521052

1053-
case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
1053+
case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) =>
10541054
val failedStage = stageIdToStage(task.stageId)
10551055
val mapStage = shuffleToMapStage(shuffleId)
10561056

@@ -1060,7 +1060,7 @@ class DAGScheduler(
10601060
if (runningStages.contains(failedStage)) {
10611061
logInfo(s"Marking $failedStage (${failedStage.name}) as failed " +
10621062
s"due to a fetch failure from $mapStage (${mapStage.name})")
1063-
markStageAsFinished(failedStage, Some("Fetch failure"))
1063+
markStageAsFinished(failedStage, Some(failureMessage))
10641064
runningStages -= failedStage
10651065
}
10661066

core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -215,7 +215,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener
215215
taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId +
216216
" STAGE_ID=" + taskEnd.stageId
217217
stageLogInfo(taskEnd.stageId, taskStatus)
218-
case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
218+
case FetchFailed(bmAddress, shuffleId, mapId, reduceId, message) =>
219219
taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" +
220220
taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" +
221221
mapId + " REDUCE_ID=" + reduceId

core/src/main/scala/org/apache/spark/shuffle/FetchFailedException.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@ package org.apache.spark.shuffle
1919

2020
import org.apache.spark.storage.BlockManagerId
2121
import org.apache.spark.{FetchFailed, TaskEndReason}
22+
import org.apache.spark.util.Utils
2223

2324
/**
2425
* Failed to fetch a shuffle block. The executor catches this exception and propagates it
@@ -30,13 +31,15 @@ private[spark] class FetchFailedException(
3031
bmAddress: BlockManagerId,
3132
shuffleId: Int,
3233
mapId: Int,
33-
reduceId: Int)
34+
reduceId: Int,
35+
message: String)
3436
extends Exception {
3537

3638
override def getMessage: String =
3739
"Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId)
3840

39-
def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId)
41+
def toTaskEndReason: TaskEndReason =
42+
FetchFailed(bmAddress, shuffleId, mapId, reduceId, message)
4043
}
4144

4245
/**
@@ -46,7 +49,7 @@ private[spark] class MetadataFetchFailedException(
4649
shuffleId: Int,
4750
reduceId: Int,
4851
message: String)
49-
extends FetchFailedException(null, shuffleId, -1, reduceId) {
52+
extends FetchFailedException(null, shuffleId, -1, reduceId, message) {
5053

5154
override def getMessage: String = message
5255
}

core/src/main/scala/org/apache/spark/shuffle/hash/BlockStoreShuffleFetcher.scala

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -19,12 +19,13 @@ package org.apache.spark.shuffle.hash
1919

2020
import scala.collection.mutable.ArrayBuffer
2121
import scala.collection.mutable.HashMap
22+
import scala.util.{Failure, Success, Try}
2223

2324
import org.apache.spark._
2425
import org.apache.spark.serializer.Serializer
2526
import org.apache.spark.shuffle.FetchFailedException
2627
import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockFetcherIterator, ShuffleBlockId}
27-
import org.apache.spark.util.CompletionIterator
28+
import org.apache.spark.util.{Utils, CompletionIterator}
2829

2930
private[hash] object BlockStoreShuffleFetcher extends Logging {
3031
def fetch[T](
@@ -52,18 +53,19 @@ private[hash] object BlockStoreShuffleFetcher extends Logging {
5253
(address, splits.map(s => (ShuffleBlockId(shuffleId, s._1, reduceId), s._2)))
5354
}
5455

55-
def unpackBlock(blockPair: (BlockId, Option[Iterator[Any]])) : Iterator[T] = {
56+
def unpackBlock(blockPair: (BlockId, Try[Iterator[Any]])) : Iterator[T] = {
5657
val blockId = blockPair._1
5758
val blockOption = blockPair._2
5859
blockOption match {
59-
case Some(block) => {
60+
case Success(block) => {
6061
block.asInstanceOf[Iterator[T]]
6162
}
62-
case None => {
63+
case Failure(e) => {
6364
blockId match {
6465
case ShuffleBlockId(shufId, mapId, _) =>
6566
val address = statuses(mapId.toInt)._1
66-
throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId)
67+
throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId,
68+
Utils.exceptionString(e))
6769
case _ =>
6870
throw new SparkException(
6971
"Failed to get block " + blockId + ", which is not a shuffle block")

core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala

Lines changed: 52 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@ package org.apache.spark.storage
2020
import java.util.concurrent.LinkedBlockingQueue
2121

2222
import scala.collection.mutable.{ArrayBuffer, HashSet, Queue}
23+
import scala.util.{Failure, Success, Try}
2324

2425
import org.apache.spark.{Logging, TaskContext}
2526
import org.apache.spark.network.{BlockFetchingListener, BlockTransferService}
@@ -54,7 +55,7 @@ final class ShuffleBlockFetcherIterator(
5455
blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
5556
serializer: Serializer,
5657
maxBytesInFlight: Long)
57-
extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging {
58+
extends Iterator[(BlockId, Try[Iterator[Any]])] with Logging {
5859

5960
import ShuffleBlockFetcherIterator._
6061

@@ -117,16 +118,18 @@ final class ShuffleBlockFetcherIterator(
117118
private[this] def cleanup() {
118119
isZombie = true
119120
// Release the current buffer if necessary
120-
if (currentResult != null && !currentResult.failed) {
121-
currentResult.buf.release()
121+
currentResult match {
122+
case SuccessFetchResult(_, _, buf) => buf.release()
123+
case _ =>
122124
}
123125

124126
// Release buffers in the results queue
125127
val iter = results.iterator()
126128
while (iter.hasNext) {
127129
val result = iter.next()
128-
if (!result.failed) {
129-
result.buf.release()
130+
result match {
131+
case SuccessFetchResult(_, _, buf) => buf.release()
132+
case _ =>
130133
}
131134
}
132135
}
@@ -149,7 +152,7 @@ final class ShuffleBlockFetcherIterator(
149152
// Increment the ref count because we need to pass this to a different thread.
150153
// This needs to be released after use.
151154
buf.retain()
152-
results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf))
155+
results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf))
153156
shuffleMetrics.remoteBytesRead += buf.size
154157
shuffleMetrics.remoteBlocksFetched += 1
155158
}
@@ -158,7 +161,7 @@ final class ShuffleBlockFetcherIterator(
158161

159162
override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
160163
logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
161-
results.put(new FetchResult(BlockId(blockId), -1, null))
164+
results.put(new FailureFetchResult(BlockId(blockId), e))
162165
}
163166
}
164167
)
@@ -229,12 +232,12 @@ final class ShuffleBlockFetcherIterator(
229232
val buf = blockManager.getBlockData(blockId)
230233
shuffleMetrics.localBlocksFetched += 1
231234
buf.retain()
232-
results.put(new FetchResult(blockId, 0, buf))
235+
results.put(new SuccessFetchResult(blockId, 0, buf))
233236
} catch {
234237
case e: Exception =>
235238
// If we see an exception, stop immediately.
236239
logError(s"Error occurred while fetching local blocks", e)
237-
results.put(new FetchResult(blockId, -1, null))
240+
results.put(new FailureFetchResult(blockId, e))
238241
return
239242
}
240243
}
@@ -265,36 +268,39 @@ final class ShuffleBlockFetcherIterator(
265268

266269
override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch
267270

268-
override def next(): (BlockId, Option[Iterator[Any]]) = {
271+
override def next(): (BlockId, Try[Iterator[Any]]) = {
269272
numBlocksProcessed += 1
270273
val startFetchWait = System.currentTimeMillis()
271274
currentResult = results.take()
272275
val result = currentResult
273276
val stopFetchWait = System.currentTimeMillis()
274277
shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait)
275-
if (!result.failed) {
276-
bytesInFlight -= result.size
278+
279+
result match {
280+
case SuccessFetchResult(_, size, _) => bytesInFlight -= size
281+
case _ =>
277282
}
278283
// Send fetch requests up to maxBytesInFlight
279284
while (fetchRequests.nonEmpty &&
280285
(bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) {
281286
sendRequest(fetchRequests.dequeue())
282287
}
283288

284-
val iteratorOpt: Option[Iterator[Any]] = if (result.failed) {
285-
None
286-
} else {
287-
val is = blockManager.wrapForCompression(result.blockId, result.buf.createInputStream())
288-
val iter = serializer.newInstance().deserializeStream(is).asIterator
289-
Some(CompletionIterator[Any, Iterator[Any]](iter, {
290-
// Once the iterator is exhausted, release the buffer and set currentResult to null
291-
// so we don't release it again in cleanup.
292-
currentResult = null
293-
result.buf.release()
294-
}))
289+
val iteratorTry: Try[Iterator[Any]] = result match {
290+
case FailureFetchResult(_, e) => Failure(e)
291+
case SuccessFetchResult(blockId, _, buf) => {
292+
val is = blockManager.wrapForCompression(blockId, buf.createInputStream())
293+
val iter = serializer.newInstance().deserializeStream(is).asIterator
294+
Success(CompletionIterator[Any, Iterator[Any]](iter, {
295+
// Once the iterator is exhausted, release the buffer and set currentResult to null
296+
// so we don't release it again in cleanup.
297+
currentResult = null
298+
buf.release()
299+
}))
300+
}
295301
}
296302

297-
(result.blockId, iteratorOpt)
303+
(result.blockId, iteratorTry)
298304
}
299305
}
300306

@@ -313,14 +319,30 @@ object ShuffleBlockFetcherIterator {
313319
}
314320

315321
/**
316-
* Result of a fetch from a remote block. A failure is represented as size == -1.
322+
* Result of a fetch from a remote block.
323+
*/
324+
trait FetchResult {
325+
val blockId: BlockId
326+
}
327+
328+
/**
329+
* Result of a fetch from a remote block successfully.
317330
* @param blockId block id
318331
* @param size estimated size of the block, used to calculate bytesInFlight.
319-
* Note that this is NOT the exact bytes. -1 if failure is present.
320-
* @param buf [[ManagedBuffer]] for the content. null is error.
332+
* Note that this is NOT the exact bytes.
333+
* @param buf [[ManagedBuffer]] for the content.
334+
*/
335+
case class SuccessFetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer)
336+
extends FetchResult {
337+
require(buf != null)
338+
require(size >= 0)
339+
}
340+
341+
/**
342+
* Result of a fetch from a remote block unsuccessfully.
343+
* @param blockId block id
344+
* @param e the failure exception
321345
*/
322-
case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) {
323-
def failed: Boolean = size == -1
324-
if (failed) assert(buf == null) else assert(buf != null)
346+
case class FailureFetchResult(blockId: BlockId, e: Throwable) extends FetchResult {
325347
}
326348
}

core/src/main/scala/org/apache/spark/util/JsonProtocol.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -279,7 +279,8 @@ private[spark] object JsonProtocol {
279279
("Block Manager Address" -> blockManagerAddress) ~
280280
("Shuffle ID" -> fetchFailed.shuffleId) ~
281281
("Map ID" -> fetchFailed.mapId) ~
282-
("Reduce ID" -> fetchFailed.reduceId)
282+
("Reduce ID" -> fetchFailed.reduceId) ~
283+
("Message" -> fetchFailed.message)
283284
case exceptionFailure: ExceptionFailure =>
284285
val stackTrace = stackTraceToJson(exceptionFailure.stackTrace)
285286
val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing)
@@ -627,7 +628,8 @@ private[spark] object JsonProtocol {
627628
val shuffleId = (json \ "Shuffle ID").extract[Int]
628629
val mapId = (json \ "Map ID").extract[Int]
629630
val reduceId = (json \ "Reduce ID").extract[Int]
630-
new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId)
631+
val message = (json \ "Message").extract[String]
632+
new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId, message)
631633
case `exceptionFailure` =>
632634
val className = (json \ "Class Name").extract[String]
633635
val description = (json \ "Description").extract[String]

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1593,7 +1593,7 @@ private[spark] object Utils extends Logging {
15931593
}
15941594

15951595
/** Return a nice string representation of the exception, including the stack trace. */
1596-
def exceptionString(e: Exception): String = {
1596+
def exceptionString(e: Throwable): String = {
15971597
if (e == null) "" else exceptionString(getFormattedClassName(e), e.getMessage, e.getStackTrace)
15981598
}
15991599

core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -431,7 +431,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
431431
// the 2nd ResultTask failed
432432
complete(taskSets(1), Seq(
433433
(Success, 42),
434-
(FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), null)))
434+
(FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)))
435435
// this will get called
436436
// blockManagerMaster.removeExecutor("exec-hostA")
437437
// ask the scheduler to try it again
@@ -461,7 +461,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
461461
// The first result task fails, with a fetch failure for the output from the first mapper.
462462
runEvent(CompletionEvent(
463463
taskSets(1).tasks(0),
464-
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0),
464+
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"),
465465
null,
466466
Map[Long, Any](),
467467
null,
@@ -472,7 +472,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
472472
// The second ResultTask fails, with a fetch failure for the output from the second mapper.
473473
runEvent(CompletionEvent(
474474
taskSets(1).tasks(0),
475-
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1),
475+
FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1, "ignored"),
476476
null,
477477
Map[Long, Any](),
478478
null,
@@ -624,7 +624,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
624624
(Success, makeMapStatus("hostC", 1))))
625625
// fail the third stage because hostA went down
626626
complete(taskSets(2), Seq(
627-
(FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null)))
627+
(FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null)))
628628
// TODO assert this:
629629
// blockManagerMaster.removeExecutor("exec-hostA")
630630
// have DAGScheduler try again
@@ -655,7 +655,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F
655655
(Success, makeMapStatus("hostB", 1))))
656656
// pretend stage 0 failed because hostA went down
657657
complete(taskSets(2), Seq(
658-
(FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0), null)))
658+
(FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0, 0, "ignored"), null)))
659659
// TODO assert this:
660660
// blockManagerMaster.removeExecutor("exec-hostA")
661661
// DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun.

core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite {
105105
for (i <- 0 until 5) {
106106
assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements")
107107
val (blockId, subIterator) = iterator.next()
108-
assert(subIterator.isDefined,
108+
assert(subIterator.isSuccess,
109109
s"iterator should have 5 elements defined but actually has $i elements")
110110

111111
// Make sure we release the buffer once the iterator is exhausted.
@@ -233,8 +233,8 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite {
233233
sem.acquire()
234234

235235
// The first block should be defined, and the last two are not defined (due to failure)
236-
assert(iterator.next()._2.isDefined === true)
237-
assert(iterator.next()._2.isDefined === false)
238-
assert(iterator.next()._2.isDefined === false)
236+
assert(iterator.next()._2.isSuccess)
237+
assert(iterator.next()._2.isFailure)
238+
assert(iterator.next()._2.isFailure)
239239
}
240240
}

0 commit comments

Comments
 (0)