Skip to content
Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ final class ShuffleBlockFetcherIterator(
private[this] val startTime = System.currentTimeMillis

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

/** Remote blocks to fetch, excluding zero-sized blocks. */
private[this] val remoteBlocks = new HashSet[BlockId]()
Expand Down Expand Up @@ -316,6 +316,7 @@ final class ShuffleBlockFetcherIterator(
* track in-memory are the ManagedBuffer references themselves.
*/
private[this] def fetchLocalBlocks() {
logDebug(s"Start fetching local blocks: ${localBlocks.mkString(", ")}")
val iter = localBlocks.iterator
while (iter.hasNext) {
val blockId = iter.next()
Expand All @@ -324,7 +325,8 @@ final class ShuffleBlockFetcherIterator(
shuffleMetrics.incLocalBlocksFetched(1)
shuffleMetrics.incLocalBytesRead(buf.size)
buf.retain()
results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId, 0, buf, false))
results.put(new SuccessFetchResult(blockId, blockManager.blockManagerId,
buf.size(), buf, false))
} catch {
case e: Exception =>
// If we see an exception, stop immediately.
Expand Down Expand Up @@ -397,7 +399,9 @@ final class ShuffleBlockFetcherIterator(
}
shuffleMetrics.incRemoteBlocksFetched(1)
}
bytesInFlight -= size
if (!localBlocks.contains(blockId)) {
bytesInFlight -= size
}
if (isNetworkReqDone) {
reqsInFlight -= 1
logDebug("Number of requests in flight " + reqsInFlight)
Expand Down Expand Up @@ -583,8 +587,8 @@ object ShuffleBlockFetcherIterator {
* Result of a fetch from a remote block successfully.
* @param blockId block id
* @param address BlockManager that the block was fetched from.
* @param size estimated size of the block, used to calculate bytesInFlight.
* Note that this is NOT the exact bytes.
* @param size estimated size of the block. Note that this is NOT the exact bytes.
* Size of remote block is used to calculate bytesInFlight.
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: documentation style

* @param buf `ManagedBuffer` for the content.
* @param isNetworkReqDone Is this the last network request for this host in this fetch request.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,63 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT
intercept[FetchFailedException] { iterator.next() }
}

test("big blocks are not checked for corruption") {
val corruptStream = mock(classOf[InputStream])
when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt"))
val corruptBuffer = mock(classOf[ManagedBuffer])
when(corruptBuffer.createInputStream()).thenReturn(corruptStream)
doReturn(10000L).when(corruptBuffer).size()

val blockManager = mock(classOf[BlockManager])
val localBmId = BlockManagerId("test-client", "test-client", 1)
doReturn(localBmId).when(blockManager).blockManagerId
doReturn(corruptBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0))
val localBlockLengths = Seq[Tuple2[BlockId, Long]](
ShuffleBlockId(0, 0, 0) -> corruptBuffer.size()
)

val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2)
val remoteBlockLengths = Seq[Tuple2[BlockId, Long]](
ShuffleBlockId(0, 1, 0) -> corruptBuffer.size()
)

val transfer = mock(classOf[BlockTransferService])
when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any()))
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you can reuse createMockTransfer to simplify this a little.

(actually, a bunch of this test code looks like it could be refactored across these tests -- but we can leave that out of this change.)

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks a lot~ Imran, I can file another pr for the refine :)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry my comment was vague -- I do think you can use createMockTransfer here, since that helper method already exists.

I was just thinking that there may be more we could clean up -- setting up the local & remote BlockManager Id, creating the ShuffleIterator, etc. seems to have a lot of boilerplate in all the tests. But let's not to do a pure refactoring to the other tests in this change.

.thenAnswer(new Answer[Unit] {
override def answer(invocation: InvocationOnMock): Unit = {
val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener]
val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]]
Future {
blocks.foreach (listener.onBlockFetchSuccess(_, corruptBuffer))
}
}
})

val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])](
(localBmId, localBlockLengths),
(remoteBmId, remoteBlockLengths)
)

val taskContext = TaskContext.empty()
val iterator = new ShuffleBlockFetcherIterator(
taskContext,
transfer,
blockManager,
blocksByAddress,
(_, in) => new LimitedInputStream(in, 10000),
2048,
Int.MaxValue,
Int.MaxValue,
Int.MaxValue,
true)
// Blocks should be returned without exceptions.
val blockSet = collection.mutable.HashSet[BlockId]()
blockSet.add(iterator.next()._1)
blockSet.add(iterator.next()._1)
assert(blockSet == collection.immutable.HashSet(
ShuffleBlockId(0, 0, 0), ShuffleBlockId(0, 1, 0)))
}

test("retry corrupt blocks (disabled)") {
val blockManager = mock(classOf[BlockManager])
val localBmId = BlockManagerId("test-client", "test-client", 1)
Expand Down