Ngone51 commented on a change in pull request #35085:
URL: https://github.com/apache/spark/pull/35085#discussion_r829115871
##########
File path:
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -294,33 +298,74 @@ class BlockManagerMasterEndpoint(
}
}.toSeq
- val removeRddBlockViaExtShuffleServiceFutures =
externalBlockStoreClient.map { shuffleClient =>
- blocksToDeleteByShuffleService.map { case (bmId, blockIds) =>
- Future[Int] {
- val numRemovedBlocks = shuffleClient.removeBlocks(
- bmId.host,
- bmId.port,
- bmId.executorId,
- blockIds.map(_.toString).toArray)
- numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds,
TimeUnit.SECONDS)
+ val removeRddBlockViaExtShuffleServiceFutures = if
(externalShuffleServiceRddFetchEnabled) {
+ externalBlockStoreClient.map { shuffleClient =>
+ blocksToDeleteByShuffleService.map { case (bmId, blockIds) =>
+ Future[Int] {
+ val numRemovedBlocks = shuffleClient.removeBlocks(
+ bmId.host,
+ bmId.port,
+ bmId.executorId,
+ blockIds.map(_.toString).toArray)
+ numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds,
TimeUnit.SECONDS)
+ }
}
- }
- }.getOrElse(Seq.empty)
+ }.getOrElse(Seq.empty)
+ } else {
+ Seq.empty
+ }
Future.sequence(removeRddFromExecutorsFutures ++
removeRddBlockViaExtShuffleServiceFutures)
}
private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = {
- // Nothing to do in the BlockManagerMasterEndpoint data structures
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
+
+ // Find all shuffle blocks on executors that are no longer running
Review comment:
Shall we unify these two cases together? e.g.,
```scala
mapOutputTracker.shuffleStatuses.get(shuffleId).map { shuffleStatus =>
val (mapStatusAtExecutor, mapStatusAtShuffleService) = {
shuffleStatus.mapStatuses.partition { mapStatus =>
blockManagerIdByExecutor.contains(mapStatus.location.executorId)
} match {
case (mapStatusAtExecutor, _) if
!externalShuffleServiceRemoveShuffleEnabled =>
(mapStatusAtExecutor, Array.empty)
case (se, ss) =>
(se, ss)
}
}
mapStatusAtExecutor.map(_.location).toSet.map {
...
} ++ mapStatusAtShuffleService.map {
...
}
}
```
This way makes us only send shuffle remove request to valid executors (not
all active executors) in the first case.
##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -94,7 +98,16 @@ private[spark] class DiskBlockManager(
} else {
val newDir = new File(localDirs(dirId), "%02x".format(subDirId))
if (!newDir.exists()) {
- Files.createDirectory(newDir.toPath)
+ val path = newDir.toPath
+ Files.createDirectory(path)
+ if (shuffleServiceRemoveShuffleEnabled) {
Review comment:
Could we only do this for shuffle files? e.g.,
```suggestion
if (shuffleServiceRemoveShuffleEnabled &&
filename.startWith("shuffle")) {
```
##########
File path:
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -294,33 +298,74 @@ class BlockManagerMasterEndpoint(
}
}.toSeq
- val removeRddBlockViaExtShuffleServiceFutures =
externalBlockStoreClient.map { shuffleClient =>
- blocksToDeleteByShuffleService.map { case (bmId, blockIds) =>
- Future[Int] {
- val numRemovedBlocks = shuffleClient.removeBlocks(
- bmId.host,
- bmId.port,
- bmId.executorId,
- blockIds.map(_.toString).toArray)
- numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds,
TimeUnit.SECONDS)
+ val removeRddBlockViaExtShuffleServiceFutures = if
(externalShuffleServiceRddFetchEnabled) {
+ externalBlockStoreClient.map { shuffleClient =>
+ blocksToDeleteByShuffleService.map { case (bmId, blockIds) =>
+ Future[Int] {
+ val numRemovedBlocks = shuffleClient.removeBlocks(
+ bmId.host,
+ bmId.port,
+ bmId.executorId,
+ blockIds.map(_.toString).toArray)
+ numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds,
TimeUnit.SECONDS)
+ }
}
- }
- }.getOrElse(Seq.empty)
+ }.getOrElse(Seq.empty)
+ } else {
+ Seq.empty
+ }
Future.sequence(removeRddFromExecutorsFutures ++
removeRddBlockViaExtShuffleServiceFutures)
}
private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = {
- // Nothing to do in the BlockManagerMasterEndpoint data structures
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
+
+ // Find all shuffle blocks on executors that are no longer running
+ val blocksToDeleteByShuffleService =
+ new mutable.HashMap[BlockManagerId, mutable.HashSet[BlockId]]
+ if (externalShuffleServiceRemoveShuffleEnabled) {
+ mapOutputTracker.shuffleStatuses.get(shuffleId).foreach { shuffleStatus
=>
+ shuffleStatus.mapStatuses.foreach { mapStatus =>
+ // Port should always be external shuffle port if external shuffle
is enabled so
+ // also check if the executor has been deallocated
+ if (mapStatus.location.port == externalShuffleServicePort &&
Review comment:
So why we still check it if it's always true?
##########
File path:
core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala
##########
@@ -41,6 +41,14 @@ trait ShuffleBlockResolver {
*/
def getBlockData(blockId: BlockId, dirs: Option[Array[String]] = None):
ManagedBuffer
+ /**
+ * Retrive a list of BlockIds for a given shuffle map. Used to delete
shuffle files
+ * from the external shuffle service after the associated executor has been
removed.
+ */
+ def getBlocksForShuffle(shuffleId: Int, mapId: Long): Seq[BlockId] = {
+ Seq.empty
+ }
Review comment:
I'd be +1 for keeping backward compatible if you want to keep this
method.
--
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]