otterc commented on a change in pull request #32140:
URL: https://github.com/apache/spark/pull/32140#discussion_r649505327
##########
File path:
core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala
##########
@@ -347,77 +355,118 @@ final class ShuffleBlockFetcherIterator(
}
}
- private[this] def partitionBlocksByFetchMode(): ArrayBuffer[FetchRequest] = {
+ /**
+ * This is called from initialize and also from the fallback which is
triggered from
+ * [[PushBasedFetchHelper]].
+ */
+ private[this] def partitionBlocksByFetchMode(
+ blocksByAddress: Iterator[(BlockManagerId, Seq[(BlockId, Long, Int)])],
+ localBlocks: mutable.LinkedHashSet[(BlockId, Int)],
+ hostLocalBlocksByExecutor: mutable.LinkedHashMap[BlockManagerId,
Seq[(BlockId, Long, Int)]],
+ mergedLocalBlocks: mutable.LinkedHashSet[BlockId]):
ArrayBuffer[FetchRequest] = {
logDebug(s"maxBytesInFlight: $maxBytesInFlight, targetRemoteRequestSize: "
+ s"$targetRemoteRequestSize, maxBlocksInFlightPerAddress:
$maxBlocksInFlightPerAddress")
- // Partition to local, host-local and remote blocks. Remote blocks are
further split into
- // FetchRequests of size at most maxBytesInFlight in order to limit the
amount of data in flight
+ // Partition to local, host-local, merged-local, remote (includes
merged-remote) blocks.
+ // Remote blocks are further split into FetchRequests of size at most
maxBytesInFlight in order
+ // to limit the amount of data in flight
val collectedRemoteRequests = new ArrayBuffer[FetchRequest]
+ val hostLocalBlocksCurrentIteration = mutable.LinkedHashSet[(BlockId,
Int)]()
var localBlockBytes = 0L
var hostLocalBlockBytes = 0L
+ var mergedLocalBlockBytes = 0L
var remoteBlockBytes = 0L
+ val prevNumBlocksToFetch = numBlocksToFetch
val fallback = FallbackStorage.FALLBACK_BLOCK_MANAGER_ID.executorId
for ((address, blockInfos) <- blocksByAddress) {
- if (Seq(blockManager.blockManagerId.executorId,
fallback).contains(address.executorId)) {
- checkBlockSizes(blockInfos)
+ checkBlockSizes(blockInfos)
+ if (pushBasedFetchHelper.isMergedShuffleBlockAddress(address)) {
+ // These are push-based merged blocks or chunks of these merged blocks.
+ if (address.host == blockManager.blockManagerId.host) {
+ val pushMergedBlockInfos = blockInfos.map(
+ info => FetchBlockInfo(info._1, info._2, info._3))
+ numBlocksToFetch += pushMergedBlockInfos.size
+ mergedLocalBlocks ++= pushMergedBlockInfos.map(info => info.blockId)
+ val size = pushMergedBlockInfos.map(_.size).sum
+ logInfo(s"Got ${pushMergedBlockInfos.size} local merged blocks " +
+ s"of size $size")
+ mergedLocalBlockBytes += size
+ } else {
+ remoteBlockBytes += blockInfos.map(_._2).sum
+ collectFetchRequests(address, blockInfos, collectedRemoteRequests)
+ }
+ } else if (
+ Seq(blockManager.blockManagerId.executorId,
fallback).contains(address.executorId)) {
val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)),
doBatchFetch)
numBlocksToFetch += mergedBlockInfos.size
localBlocks ++= mergedBlockInfos.map(info => (info.blockId,
info.mapIndex))
localBlockBytes += mergedBlockInfos.map(_.size).sum
} else if (blockManager.hostLocalDirManager.isDefined &&
address.host == blockManager.blockManagerId.host) {
- checkBlockSizes(blockInfos)
val mergedBlockInfos = mergeContinuousShuffleBlockIdsIfNeeded(
blockInfos.map(info => FetchBlockInfo(info._1, info._2, info._3)),
doBatchFetch)
numBlocksToFetch += mergedBlockInfos.size
val blocksForAddress =
mergedBlockInfos.map(info => (info.blockId, info.size,
info.mapIndex))
hostLocalBlocksByExecutor += address -> blocksForAddress
- hostLocalBlocks ++= blocksForAddress.map(info => (info._1, info._3))
+ hostLocalBlocksCurrentIteration ++= blocksForAddress.map(info =>
(info._1, info._3))
hostLocalBlockBytes += mergedBlockInfos.map(_.size).sum
} else {
remoteBlockBytes += blockInfos.map(_._2).sum
collectFetchRequests(address, blockInfos, collectedRemoteRequests)
}
}
val numRemoteBlocks = collectedRemoteRequests.map(_.blocks.size).sum
- val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes
- assert(numBlocksToFetch == localBlocks.size + hostLocalBlocks.size +
numRemoteBlocks,
- s"The number of non-empty blocks $numBlocksToFetch doesn't equal to the
number of local " +
- s"blocks ${localBlocks.size} + the number of host-local blocks
${hostLocalBlocks.size} " +
- s"+ the number of remote blocks ${numRemoteBlocks}.")
- logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)})
non-empty blocks " +
- s"including ${localBlocks.size}
(${Utils.bytesToString(localBlockBytes)}) local and " +
- s"${hostLocalBlocks.size} (${Utils.bytesToString(hostLocalBlockBytes)})
" +
- s"host-local and $numRemoteBlocks
(${Utils.bytesToString(remoteBlockBytes)}) remote blocks")
+ val totalBytes = localBlockBytes + remoteBlockBytes + hostLocalBlockBytes +
+ mergedLocalBlockBytes
+ val blocksToFetchCurrentIteration = numBlocksToFetch - prevNumBlocksToFetch
+ assert(blocksToFetchCurrentIteration == localBlocks.size +
+ hostLocalBlocksCurrentIteration.size + numRemoteBlocks +
mergedLocalBlocks.size,
+ s"The number of non-empty blocks $blocksToFetchCurrentIteration doesn't
equal to " +
+ s"the number of local blocks ${localBlocks.size} + " +
+ s"the number of host-local blocks
${hostLocalBlocksCurrentIteration.size} " +
+ s"the number of merged-local blocks ${mergedLocalBlocks.size} " +
+ s"+ the number of remote blocks ${numRemoteBlocks} ")
+ logInfo(s"Getting $blocksToFetchCurrentIteration " +
+ s"(${Utils.bytesToString(totalBytes)}) non-empty blocks including " +
+ s"${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local
and " +
+ s"${hostLocalBlocksCurrentIteration.size}
(${Utils.bytesToString(hostLocalBlockBytes)}) " +
+ s"host-local and ${mergedLocalBlocks.size}
(${Utils.bytesToString(mergedLocalBlockBytes)}) " +
+ s"local merged and $numRemoteBlocks
(${Utils.bytesToString(remoteBlockBytes)}) " +
+ s"remote blocks")
+ if (hostLocalBlocksCurrentIteration.nonEmpty) {
+ this.hostLocalBlocks ++= hostLocalBlocksCurrentIteration
+ }
collectedRemoteRequests
}
private def createFetchRequest(
blocks: Seq[FetchBlockInfo],
- address: BlockManagerId): FetchRequest = {
+ address: BlockManagerId,
+ forMergedMetas: Boolean = false): FetchRequest = {
logDebug(s"Creating fetch request of ${blocks.map(_.size).sum} at $address
"
+ s"with ${blocks.size} blocks")
- FetchRequest(address, blocks)
+ FetchRequest(address, blocks, forMergedMetas)
}
private def createFetchRequests(
curBlocks: Seq[FetchBlockInfo],
address: BlockManagerId,
isLast: Boolean,
- collectedRemoteRequests: ArrayBuffer[FetchRequest]): Seq[FetchBlockInfo]
= {
- val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks,
doBatchFetch)
+ collectedRemoteRequests: ArrayBuffer[FetchRequest],
+ enableBatchFetch: Boolean,
+ forMergedMetas: Boolean = false): Seq[FetchBlockInfo] = {
+ val mergedBlocks = mergeContinuousShuffleBlockIdsIfNeeded(curBlocks,
enableBatchFetch)
Review comment:
> Is mergeContinuousShuffleBlockIdsIfNeeded relevant for merged
blocks/chunks ?
No, it is not relevant for merged blocks/chunks. For both merged
blocks/chunks, I am passing `enabledBatchFetch = false` so
`mergeContinuousShuffleBlockIdsIfNeeded` returns the passed in blocks.
I am not seeing any side effects of reusing this method for merged
blocks/chunks. IIUC, this method enforces the limit of
`maxBlocksInFlightPerAddress` for a FetchRequest and is the one that modifies
`numBlocksToFetch` for remote requests.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]