-
Notifications
You must be signed in to change notification settings - Fork 28.7k
[SPARK-4163][Core][WebUI] Send the fetch failure message back to Web UI #3032
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 all commits
a3bca65
0c07d1f
62103fd
b88c919
d51b0b6
316767d
4e946f7
f7e1faf
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 |
---|---|---|
|
@@ -1050,7 +1050,7 @@ class DAGScheduler( | |
logInfo("Resubmitted " + task + ", so marking it as still running") | ||
stage.pendingTasks += task | ||
|
||
case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => | ||
case FetchFailed(bmAddress, shuffleId, mapId, reduceId, failureMessage) => | ||
val failedStage = stageIdToStage(task.stageId) | ||
val mapStage = shuffleToMapStage(shuffleId) | ||
|
||
|
@@ -1060,7 +1060,7 @@ class DAGScheduler( | |
if (runningStages.contains(failedStage)) { | ||
logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + | ||
s"due to a fetch failure from $mapStage (${mapStage.name})") | ||
markStageAsFinished(failedStage, Some("Fetch failure")) | ||
markStageAsFinished(failedStage, Some("Fetch failure: " + failureMessage)) | ||
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. I was wrong here, I didn't realize that the failureMessage includes the "FetchFailedException: ", which makes the prepending of "Fetch failure:" redundant. Please go ahead and remove it again, sorry for making you do extra work! 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. If I I revert to the Utils.exceptionString() way, |
||
runningStages -= failedStage | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,6 +20,7 @@ package org.apache.spark.storage | |
import java.util.concurrent.LinkedBlockingQueue | ||
|
||
import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} | ||
import scala.util.{Failure, Success, Try} | ||
|
||
import org.apache.spark.{Logging, TaskContext} | ||
import org.apache.spark.network.{BlockFetchingListener, BlockTransferService} | ||
|
@@ -54,7 +55,7 @@ final class ShuffleBlockFetcherIterator( | |
blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], | ||
serializer: Serializer, | ||
maxBytesInFlight: Long) | ||
extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { | ||
extends Iterator[(BlockId, Try[Iterator[Any]])] with Logging { | ||
|
||
import ShuffleBlockFetcherIterator._ | ||
|
||
|
@@ -117,16 +118,18 @@ final class ShuffleBlockFetcherIterator( | |
private[this] def cleanup() { | ||
isZombie = true | ||
// Release the current buffer if necessary | ||
if (currentResult != null && !currentResult.failed) { | ||
currentResult.buf.release() | ||
currentResult match { | ||
case SuccessFetchResult(_, _, buf) => buf.release() | ||
case _ => | ||
} | ||
|
||
// Release buffers in the results queue | ||
val iter = results.iterator() | ||
while (iter.hasNext) { | ||
val result = iter.next() | ||
if (!result.failed) { | ||
result.buf.release() | ||
result match { | ||
case SuccessFetchResult(_, _, buf) => buf.release() | ||
case _ => | ||
} | ||
} | ||
} | ||
|
@@ -149,7 +152,7 @@ final class ShuffleBlockFetcherIterator( | |
// Increment the ref count because we need to pass this to a different thread. | ||
// This needs to be released after use. | ||
buf.retain() | ||
results.put(new FetchResult(BlockId(blockId), sizeMap(blockId), buf)) | ||
results.put(new SuccessFetchResult(BlockId(blockId), sizeMap(blockId), buf)) | ||
shuffleMetrics.remoteBytesRead += buf.size | ||
shuffleMetrics.remoteBlocksFetched += 1 | ||
} | ||
|
@@ -158,7 +161,7 @@ final class ShuffleBlockFetcherIterator( | |
|
||
override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { | ||
logError(s"Failed to get block(s) from ${req.address.host}:${req.address.port}", e) | ||
results.put(new FetchResult(BlockId(blockId), -1, null)) | ||
results.put(new FailureFetchResult(BlockId(blockId), e)) | ||
} | ||
} | ||
) | ||
|
@@ -229,12 +232,12 @@ final class ShuffleBlockFetcherIterator( | |
val buf = blockManager.getBlockData(blockId) | ||
shuffleMetrics.localBlocksFetched += 1 | ||
buf.retain() | ||
results.put(new FetchResult(blockId, 0, buf)) | ||
results.put(new SuccessFetchResult(blockId, 0, buf)) | ||
} catch { | ||
case e: Exception => | ||
// If we see an exception, stop immediately. | ||
logError(s"Error occurred while fetching local blocks", e) | ||
results.put(new FetchResult(blockId, -1, null)) | ||
results.put(new FailureFetchResult(blockId, e)) | ||
return | ||
} | ||
} | ||
|
@@ -265,36 +268,39 @@ final class ShuffleBlockFetcherIterator( | |
|
||
override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch | ||
|
||
override def next(): (BlockId, Option[Iterator[Any]]) = { | ||
override def next(): (BlockId, Try[Iterator[Any]]) = { | ||
numBlocksProcessed += 1 | ||
val startFetchWait = System.currentTimeMillis() | ||
currentResult = results.take() | ||
val result = currentResult | ||
val stopFetchWait = System.currentTimeMillis() | ||
shuffleMetrics.fetchWaitTime += (stopFetchWait - startFetchWait) | ||
if (!result.failed) { | ||
bytesInFlight -= result.size | ||
|
||
result match { | ||
case SuccessFetchResult(_, size, _) => bytesInFlight -= size | ||
case _ => | ||
} | ||
// Send fetch requests up to maxBytesInFlight | ||
while (fetchRequests.nonEmpty && | ||
(bytesInFlight == 0 || bytesInFlight + fetchRequests.front.size <= maxBytesInFlight)) { | ||
sendRequest(fetchRequests.dequeue()) | ||
} | ||
|
||
val iteratorOpt: Option[Iterator[Any]] = if (result.failed) { | ||
None | ||
} else { | ||
val is = blockManager.wrapForCompression(result.blockId, result.buf.createInputStream()) | ||
val iter = serializer.newInstance().deserializeStream(is).asIterator | ||
Some(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 | ||
result.buf.release() | ||
})) | ||
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() | ||
})) | ||
} | ||
} | ||
|
||
(result.blockId, iteratorOpt) | ||
(result.blockId, iteratorTry) | ||
} | ||
} | ||
|
||
|
@@ -313,14 +319,30 @@ object ShuffleBlockFetcherIterator { | |
} | ||
|
||
/** | ||
* Result of a fetch from a remote block. A failure is represented as size == -1. | ||
* Result of a fetch from a remote block. | ||
*/ | ||
private[storage] sealed trait FetchResult { | ||
val blockId: BlockId | ||
} | ||
|
||
/** | ||
* Result of a fetch from a remote block successfully. | ||
* @param blockId block id | ||
* @param size estimated size of the block, used to calculate bytesInFlight. | ||
* Note that this is NOT the exact bytes. -1 if failure is present. | ||
* @param buf [[ManagedBuffer]] for the content. null is error. | ||
* Note that this is NOT the exact bytes. | ||
* @param buf [[ManagedBuffer]] for the content. | ||
*/ | ||
case class FetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) { | ||
def failed: Boolean = size == -1 | ||
if (failed) assert(buf == null) else assert(buf != null) | ||
private[storage] case class SuccessFetchResult(blockId: BlockId, size: Long, buf: ManagedBuffer) | ||
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. Just to make sure we're on absolutely the same page, Adding 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. Got it. Thank you. |
||
extends FetchResult { | ||
require(buf != null) | ||
require(size >= 0) | ||
} | ||
|
||
/** | ||
* Result of a fetch from a remote block unsuccessfully. | ||
* @param blockId block id | ||
* @param e the failure exception | ||
*/ | ||
private[storage] case class FailureFetchResult(blockId: BlockId, e: Throwable) | ||
extends FetchResult | ||
} |
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.
one day we'll use proper subclassing to get rid of this silly case too