Skip to content

Commit 76386e1

Browse files
zsxwingaarondav
authored andcommitted
[SPARK-4163][Core][WebUI] Send the fetch failure message back to Web UI
This is a PR to send the fetch failure message back to Web UI. Before: ![f1](https://cloud.githubusercontent.com/assets/1000778/4856595/1f036c80-60be-11e4-956f-335147fbccb7.png) ![f2](https://cloud.githubusercontent.com/assets/1000778/4856596/1f11cbea-60be-11e4-8fe9-9f9b2b35c884.png) After (Please ignore the meaning of exception, I threw it in the code directly because it's hard to simulate a fetch failure): ![e1](https://cloud.githubusercontent.com/assets/1000778/4856600/2657ea38-60be-11e4-9f2d-d56c5f900f10.png) ![e2](https://cloud.githubusercontent.com/assets/1000778/4856601/26595008-60be-11e4-912b-2744af786991.png) Author: zsxwing <[email protected]> Closes #3032 from zsxwing/SPARK-4163 and squashes the following commits: f7e1faf [zsxwing] Discard changes for FetchFailedException and minor modification 4e946f7 [zsxwing] Add e as the cause of SparkException 316767d [zsxwing] Add private[storage] to FetchResult d51b0b6 [zsxwing] Set e as the cause of FetchFailedException b88c919 [zsxwing] Use 'private[storage]' for case classes instead of 'sealed' 62103fd [zsxwing] Update as per review 0c07d1f [zsxwing] Backward-compatible support a3bca65 [zsxwing] Send the fetch failure message back to Web UI
1 parent 001acc4 commit 76386e1

File tree

12 files changed

+92
-65
lines changed

12 files changed

+92
-65
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
@@ -1053,7 +1053,7 @@ class DAGScheduler(
10531053
logInfo("Resubmitted " + task + ", so marking it as still running")
10541054
stage.pendingTasks += task
10551055

1056-
case FetchFailed(bmAddress, shuffleId, mapId, reduceId) =>
1056+
case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) =>
10571057
val failedStage = stageIdToStage(task.stageId)
10581058
val mapStage = shuffleToMapStage(shuffleId)
10591059

@@ -1063,7 +1063,7 @@ class DAGScheduler(
10631063
if (runningStages.contains(failedStage)) {
10641064
logInfo(s"Marking $failedStage (${failedStage.name}) as failed " +
10651065
s"due to a fetch failure from $mapStage (${mapStage.name})")
1066-
markStageAsFinished(failedStage, Some("Fetch failure"))
1066+
markStageAsFinished(failedStage, Some("Fetch failure: " + failureMessage))
10671067
runningStages -= failedStage
10681068
}
10691069

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 & 10 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,11 @@ private[spark] class FetchFailedException(
3031
bmAddress: BlockManagerId,
3132
shuffleId: Int,
3233
mapId: Int,
33-
reduceId: Int)
34-
extends Exception {
35-
36-
override def getMessage: String =
37-
"Fetch failed: %s %d %d %d".format(bmAddress, shuffleId, mapId, reduceId)
34+
reduceId: Int,
35+
message: String)
36+
extends Exception(message) {
3837

39-
def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId)
38+
def toTaskEndReason: TaskEndReason = FetchFailed(bmAddress, shuffleId, mapId, reduceId, message)
4039
}
4140

4241
/**
@@ -46,7 +45,4 @@ private[spark] class MetadataFetchFailedException(
4645
shuffleId: Int,
4746
reduceId: Int,
4847
message: String)
49-
extends FetchFailedException(null, shuffleId, -1, reduceId) {
50-
51-
override def getMessage: String = message
52-
}
48+
extends FetchFailedException(null, shuffleId, -1, reduceId, message)

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

Lines changed: 8 additions & 6 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.{CompletionIterator, Utils}
2829

2930
private[hash] object BlockStoreShuffleFetcher extends Logging {
3031
def fetch[T](
@@ -52,21 +53,22 @@ 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(
69-
"Failed to get block " + blockId + ", which is not a shuffle block")
71+
"Failed to get block " + blockId + ", which is not a shuffle block", e)
7072
}
7173
}
7274
}

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.BlockTransferService
@@ -55,7 +56,7 @@ final class ShuffleBlockFetcherIterator(
5556
blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])],
5657
serializer: Serializer,
5758
maxBytesInFlight: Long)
58-
extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging {
59+
extends Iterator[(BlockId, Try[Iterator[Any]])] with Logging {
5960

6061
import ShuffleBlockFetcherIterator._
6162

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

125127
// Release buffers in the results queue
126128
val iter = results.iterator()
127129
while (iter.hasNext) {
128130
val result = iter.next()
129-
if (!result.failed) {
130-
result.buf.release()
131+
result match {
132+
case SuccessFetchResult(_, _, buf) => buf.release()
133+
case _ =>
131134
}
132135
}
133136
}
@@ -151,7 +154,7 @@ final class ShuffleBlockFetcherIterator(
151154
// Increment the ref count because we need to pass this to a different thread.
152155
// This needs to be released after use.
153156
buf.retain()
154-
results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf))
157+
results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf))
155158
shuffleMetrics.remoteBytesRead += buf.size
156159
shuffleMetrics.remoteBlocksFetched += 1
157160
}
@@ -160,7 +163,7 @@ final class ShuffleBlockFetcherIterator(
160163

161164
override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = {
162165
logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e)
163-
results.put(new FetchResult(BlockId(blockId), -1, null))
166+
results.put(new FailureFetchResult(BlockId(blockId), e))
164167
}
165168
}
166169
)
@@ -231,12 +234,12 @@ final class ShuffleBlockFetcherIterator(
231234
val buf = blockManager.getBlockData(blockId)
232235
shuffleMetrics.localBlocksFetched += 1
233236
buf.retain()
234-
results.put(new FetchResult(blockId, 0, buf))
237+
results.put(new SuccessFetchResult(blockId, 0, buf))
235238
} catch {
236239
case e: Exception =>
237240
// If we see an exception, stop immediately.
238241
logError(s"Error occurred while fetching local blocks", e)
239-
results.put(new FetchResult(blockId, -1, null))
242+
results.put(new FailureFetchResult(blockId, e))
240243
return
241244
}
242245
}
@@ -267,36 +270,39 @@ final class ShuffleBlockFetcherIterator(
267270

268271
override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch
269272

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

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

299-
(result.blockId, iteratorOpt)
305+
(result.blockId, iteratorTry)
300306
}
301307
}
302308

@@ -315,14 +321,30 @@ object ShuffleBlockFetcherIterator {
315321
}
316322

317323
/**
318-
* Result of a fetch from a remote block. A failure is represented as size == -1.
324+
* Result of a fetch from a remote block.
325+
*/
326+
private[storage] sealed trait FetchResult {
327+
val blockId: BlockId
328+
}
329+
330+
/**
331+
* Result of a fetch from a remote block successfully.
319332
* @param blockId block id
320333
* @param size estimated size of the block, used to calculate bytesInFlight.
321-
* Note that this is NOT the exact bytes. -1 if failure is present.
322-
* @param buf [[ManagedBuffer]] for the content. null is error.
334+
* Note that this is NOT the exact bytes.
335+
* @param buf [[ManagedBuffer]] for the content.
323336
*/
324-
case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) {
325-
def failed: Boolean = size == -1
326-
if (failed) assert(buf == null) else assert(buf != null)
337+
private[storage] case class SuccessFetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer)
338+
extends FetchResult {
339+
require(buf != null)
340+
require(size >= 0)
327341
}
342+
343+
/**
344+
* Result of a fetch from a remote block unsuccessfully.
345+
* @param blockId block id
346+
* @param e the failure exception
347+
*/
348+
private[storage] case class FailureFetchResult(blockId: BlockId, e: Throwable)
349+
extends FetchResult
328350
}

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

Lines changed: 5 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)
@@ -629,7 +630,9 @@ private[spark] object JsonProtocol {
629630
val shuffleId = (json \ "Shuffle ID").extract[Int]
630631
val mapId = (json \ "Map ID").extract[Int]
631632
val reduceId = (json \ "Reduce ID").extract[Int]
632-
new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId)
633+
val message = Utils.jsonOption(json \ "Message").map(_.extract[String])
634+
new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId,
635+
message.getOrElse("Unknown reason"))
633636
case `exceptionFailure` =>
634637
val className = (json \ "Class Name").extract[String]
635638
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
@@ -1597,7 +1597,7 @@ private[spark] object Utils extends Logging {
15971597
}
15981598

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

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
@@ -102,7 +102,7 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite {
102102
for (i <- 0 until 5) {
103103
assert(iterator.hasNext, s"iterator should have 5 elements but actually has $i elements")
104104
val (blockId, subIterator) = iterator.next()
105-
assert(subIterator.isDefined,
105+
assert(subIterator.isSuccess,
106106
s"iterator should have 5 elements defined but actually has $i elements")
107107

108108
// Make sure we release the buffer once the iterator is exhausted.
@@ -230,8 +230,8 @@ class ShuffleBlockFetcherIteratorSuite extends FunSuite {
230230
sem.acquire()
231231

232232
// The first block should be defined, and the last two are not defined (due to failure)
233-
assert(iterator.next()._2.isDefined === true)
234-
assert(iterator.next()._2.isDefined === false)
235-
assert(iterator.next()._2.isDefined === false)
233+
assert(iterator.next()._2.isSuccess)
234+
assert(iterator.next()._2.isFailure)
235+
assert(iterator.next()._2.isFailure)
236236
}
237237
}

0 commit comments

Comments
 (0)