Kimahriman commented on a change in pull request #35085:
URL: https://github.com/apache/spark/pull/35085#discussion_r778444925
##########
File path:
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -311,16 +313,46 @@ class BlockManagerMasterEndpoint(
}
private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = {
- // Nothing to do in the BlockManagerMasterEndpoint data structures
+ // Find all shuffle blocks on executors that are no longer running
+ val blocksToDeleteByShuffleService =
+ new mutable.HashMap[BlockManagerId, mutable.HashSet[BlockId]]
+ mapOutputTracker.shuffleStatuses.get(shuffleId).map { shuffleStatus =>
+ shuffleStatus.mapStatuses
+ .filter(_.location.port == externalShuffleServicePort)
+ .foreach { mapStatus =>
+ val blocks =
blocksToDeleteByShuffleService.getOrElseUpdate(mapStatus.location,
+ new mutable.HashSet[BlockId])
+ val blocksToDel =
+ shuffleManager.shuffleBlockResolver.getBlocksForShuffle(shuffleId,
mapStatus.mapId)
+ blocksToDel.foreach(blocks.add(_))
+ }
+ }
+
val removeMsg = RemoveShuffle(shuffleId)
- Future.sequence(
- blockManagerInfo.values.map { bm =>
- bm.storageEndpoint.ask[Boolean](removeMsg).recover {
- // use false as default value means no shuffle data were removed
- handleBlockRemovalFailure("shuffle", shuffleId.toString,
bm.blockManagerId, false)
+ val removeShuffleFromExecutorsFutures = blockManagerInfo.values.map { bm =>
+ bm.storageEndpoint.ask[Boolean](removeMsg).recover {
+ // use false as default value means no shuffle data were removed
+ handleBlockRemovalFailure("shuffle", shuffleId.toString,
bm.blockManagerId, false)
+ }
+ }.toSeq
+
+ val removeShuffleBlockViaExtShuffleServiceFutures =
+ externalBlockStoreClient.map { shuffleClient =>
+ blocksToDeleteByShuffleService.map { case (bmId, blockIds) =>
Review comment:
Add a check to ignore things where the executor is still alive
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]