Skip to content

Commit 110c851

Browse files
author
jx158167
committed
[SPARK-23524] Big local shuffle blocks should not be checked for corruption.
1 parent 8077bb0 commit 110c851

File tree

2 files changed

+64
-3
lines changed

2 files changed

+64
-3
lines changed

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

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -90,7 +90,7 @@ final class ShuffleBlockFetcherIterator(
9090
private[this] val startTime = System.currentTimeMillis
9191

9292
/** Local blocks to fetch, excluding zero-sized blocks. */
93-
private[this] val localBlocks = new ArrayBuffer[BlockId]()
93+
private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]()
9494

9595
/** Remote blocks to fetch, excluding zero-sized blocks. */
9696
private[this] val remoteBlocks = new HashSet[BlockId]()
@@ -316,6 +316,7 @@ final class ShuffleBlockFetcherIterator(
316316
* track in-memory are the ManagedBuffer references themselves.
317317
*/
318318
private[this] def fetchLocalBlocks() {
319+
logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}")
319320
val iter = localBlocks.iterator
320321
while (iter.hasNext) {
321322
val blockId = iter.next()
@@ -324,7 +325,8 @@ final class ShuffleBlockFetcherIterator(
324325
shuffleMetrics.incLocalBlocksFetched(1)
325326
shuffleMetrics.incLocalBytesRead(buf.size)
326327
buf.retain()
327-
results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, 0, buf, false))
328+
results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId,
329+
buf.size(), buf, false))
328330
} catch {
329331
case e: Exception =>
330332
// If we see an exception, stop immediately.
@@ -397,7 +399,9 @@ final class ShuffleBlockFetcherIterator(
397399
}
398400
shuffleMetrics.incRemoteBlocksFetched(1)
399401
}
400-
bytesInFlight -= size
402+
if (!localBlocks.contains(blockId)) {
403+
bytesInFlight -= size
404+
}
401405
if (isNetworkReqDone) {
402406
reqsInFlight -= 1
403407
logDebug("Number of requests in flight " + reqsInFlight)

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

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -352,6 +352,63 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
352352
intercept[FetchFailedException] { iterator.next() }
353353
}
354354

355+
test("big corrupt blocks will not be retiried") {
356+
val corruptStream = mock(classOf[InputStream])
357+
when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt"))
358+
val corruptBuffer = mock(classOf[ManagedBuffer])
359+
when(corruptBuffer.createInputStream()).thenReturn(corruptStream)
360+
doReturn(10000L).when(corruptBuffer).size()
361+
362+
val blockManager = mock(classOf[BlockManager])
363+
val localBmId = BlockManagerId("test-client", "test-client", 1)
364+
doReturn(localBmId).when(blockManager).blockManagerId
365+
doReturn(corruptBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0))
366+
val localBlockLengths = Seq[Tuple2[BlockId, Long]](
367+
ShuffleBlockId(0, 0, 0) -> corruptBuffer.size()
368+
)
369+
370+
val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
371+
val remoteBlockLengths = Seq[Tuple2[BlockId, Long]](
372+
ShuffleBlockId(0, 1, 0) -> corruptBuffer.size()
373+
)
374+
375+
val transfer = mock(classOf[BlockTransferService])
376+
when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
377+
.thenAnswer(new Answer[Unit] {
378+
override def answer(invocation: InvocationOnMock): Unit = {
379+
val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
380+
val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]]
381+
Future {
382+
blocks.foreach (listener.onBlockFetchSuccess(_, corruptBuffer))
383+
}
384+
}
385+
})
386+
387+
val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
388+
(localBmId, localBlockLengths),
389+
(remoteBmId, remoteBlockLengths)
390+
)
391+
392+
val taskContext = TaskContext.empty()
393+
val iterator = new ShuffleBlockFetcherIterator(
394+
taskContext,
395+
transfer,
396+
blockManager,
397+
blocksByAddress,
398+
(_, in) => new LimitedInputStream(in, 10000),
399+
2048,
400+
Int.MaxValue,
401+
Int.MaxValue,
402+
Int.MaxValue,
403+
true)
404+
// Blocks should be returned without exceptions.
405+
val blockSet = collection.mutable.HashSet[BlockId]()
406+
blockSet.add(iterator.next()._1)
407+
blockSet.add(iterator.next()._1)
408+
assert(blockSet == collection.immutable.HashSet(
409+
ShuffleBlockId(0, 0, 0), ShuffleBlockId(0, 1, 0)))
410+
}
411+
355412
test("retry corrupt blocks (disabled)") {
356413
val blockManager = mock(classOf[BlockManager])
357414
val localBmId = BlockManagerId("test-client", "test-client", 1)

0 commit comments

Comments
 (0)