-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-4105] retry the fetch or stage if shuffle block is corrupt #15923
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 1 commit
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 |
|---|---|---|
|
|
@@ -17,19 +17,22 @@ | |
|
|
||
| package org.apache.spark.storage | ||
|
|
||
| import java.io.InputStream | ||
| import java.io.{InputStream, IOException} | ||
| import java.nio.ByteBuffer | ||
| import java.util.concurrent.LinkedBlockingQueue | ||
| import javax.annotation.concurrent.GuardedBy | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} | ||
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.spark.{SparkException, TaskContext} | ||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.network.buffer.ManagedBuffer | ||
| import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} | ||
| import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient} | ||
| import org.apache.spark.shuffle.FetchFailedException | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.util.io.{ChunkedByteBufferInputStream, ChunkedByteBufferOutputStream} | ||
|
|
||
| /** | ||
| * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block | ||
|
|
@@ -56,6 +59,7 @@ final class ShuffleBlockFetcherIterator( | |
| shuffleClient: ShuffleClient, | ||
|
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. Could you update the Scaladoc to document the two new parameters here? I understand what
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. done |
||
| blockManager: BlockManager, | ||
| blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], | ||
| streamWrapper: (BlockId, InputStream) => InputStream, | ||
| maxBytesInFlight: Long, | ||
| maxReqsInFlight: Int) | ||
| extends Iterator[(BlockId, InputStream)] with Logging { | ||
|
|
@@ -108,6 +112,9 @@ final class ShuffleBlockFetcherIterator( | |
| /** Current number of requests in flight */ | ||
| private[this] var reqsInFlight = 0 | ||
|
|
||
| /** The blocks that can't be decompressed successfully */ | ||
|
||
| private[this] val corruptedBlocks = mutable.HashSet[String]() | ||
|
|
||
| private[this] val shuffleMetrics = context.taskMetrics().createTempShuffleReadMetrics() | ||
|
|
||
| /** | ||
|
|
@@ -305,35 +312,82 @@ final class ShuffleBlockFetcherIterator( | |
| */ | ||
| override def next(): (BlockId, InputStream) = { | ||
| numBlocksProcessed += 1 | ||
| val startFetchWait = System.currentTimeMillis() | ||
| currentResult = results.take() | ||
| val result = currentResult | ||
| val stopFetchWait = System.currentTimeMillis() | ||
| shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) | ||
|
|
||
| result match { | ||
| case SuccessFetchResult(_, address, size, buf, isNetworkReqDone) => | ||
| if (address != blockManager.blockManagerId) { | ||
| shuffleMetrics.incRemoteBytesRead(buf.size) | ||
| shuffleMetrics.incRemoteBlocksFetched(1) | ||
| } | ||
| bytesInFlight -= size | ||
| if (isNetworkReqDone) { | ||
| reqsInFlight -= 1 | ||
| logDebug("Number of requests in flight " + reqsInFlight) | ||
| } | ||
| case _ => | ||
| var result: FetchResult = null | ||
|
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. add documentation explaining what's going on here.
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. btw is there a way to refactor this function so it is testable? i do worry some of the logic here won't be tested at all. |
||
| var input: InputStream = null | ||
| while (result == null) { | ||
| val startFetchWait = System.currentTimeMillis() | ||
| result = results.take() | ||
| val stopFetchWait = System.currentTimeMillis() | ||
| shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) | ||
|
|
||
| result match { | ||
| case SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => | ||
| if (address != blockManager.blockManagerId) { | ||
| shuffleMetrics.incRemoteBytesRead(buf.size) | ||
| shuffleMetrics.incRemoteBlocksFetched(1) | ||
| } | ||
| bytesInFlight -= size | ||
| if (isNetworkReqDone) { | ||
| reqsInFlight -= 1 | ||
| logDebug("Number of requests in flight " + reqsInFlight) | ||
| } | ||
|
|
||
| val in = try { | ||
| buf.createInputStream() | ||
| } catch { | ||
| // The exception could only be throwed by local shuffle block | ||
| case e: IOException if buf.isInstanceOf[FileSegmentManagedBuffer] => | ||
| logError("Failed to create input stream from local block", e) | ||
| buf.release() | ||
| result = FailureFetchResult(blockId, address, e) | ||
| null | ||
| } | ||
| if (in != null) { | ||
| input = streamWrapper(blockId, in) | ||
| // Only copy the stream if it's wrapped by compression or encryption, also the size of | ||
| // block is small (the decompressed block is smaller than maxBytesInFlight) | ||
| if (!input.eq(in) && size < maxBytesInFlight / 3) { | ||
| val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) | ||
| try { | ||
| // Decompress the whole block at once to detect any corruption, which could increase | ||
| // the memory usage tne potential increase the chance of OOM. | ||
| // TODO: manage the memory used here, and spill it into disk in case of OOM. | ||
| Utils.copyStream(input, out) | ||
| input = out.toChunkedByteBuffer.toInputStream(true) | ||
| } catch { | ||
| case e: IOException => | ||
| buf.release() | ||
| if (buf.isInstanceOf[FileSegmentManagedBuffer] | ||
| || corruptedBlocks.contains(blockId.toString)) { | ||
| result = FailureFetchResult(blockId, address, e) | ||
| } else { | ||
| logWarning(s"got an corrupted block $blockId from $address, fetch again") | ||
| fetchRequests += FetchRequest(address, Array((blockId, size))) | ||
| result = null | ||
| } | ||
| } finally { | ||
| // TODO: release the buf here (earlier) | ||
| in.close() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| case _ => | ||
| } | ||
|
|
||
| // Send fetch requests up to maxBytesInFlight | ||
| fetchUpToMaxBytes() | ||
| } | ||
| // Send fetch requests up to maxBytesInFlight | ||
| fetchUpToMaxBytes() | ||
| currentResult = result | ||
|
|
||
| result match { | ||
| case FailureFetchResult(blockId, address, e) => | ||
| throwFetchFailedException(blockId, address, e) | ||
|
|
||
| case SuccessFetchResult(blockId, address, _, buf, _) => | ||
| try { | ||
| (result.blockId, new BufferReleasingInputStream(buf.createInputStream(), this)) | ||
| (result.blockId, new BufferReleasingInputStream(input, this)) | ||
| } catch { | ||
| case NonFatal(t) => | ||
| throwFetchFailedException(blockId, address, t) | ||
|
|
||
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.
seems
ChunkedByteBufferInputStreamis not used here.