mridulm commented on a change in pull request #35085:
URL: https://github.com/apache/spark/pull/35085#discussion_r785395210



##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -311,16 +313,49 @@ 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 =>

Review comment:
       `map` -> `foreach`

##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -311,16 +313,49 @@ 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.foreach { mapStatus =>
+        // Port should always be external shuffle port if external shuffle is 
enabled
+        val isShufflePort = mapStatus.location.port == 
externalShuffleServicePort

Review comment:
       When disabled, it will default to `7337` which could be reused by an 
executor.
   Add check for `externalBlockStoreClient.isDefined` as well.
   ```suggestion
           val isShufflePort = externalBlockStoreClient.isDefined && 
mapStatus.location.port == externalShuffleServicePort
   ```
   
   Better still, add `externalBlockStoreClient.isDefined` around the entire 
block.

##########
File path: 
core/src/main/java/org/apache/spark/shuffle/sort/io/LocalDiskShuffleMapOutputWriter.java
##########
@@ -88,6 +88,8 @@ public ShufflePartitionWriter getPartitionWriter(int 
reducePartitionId) throws I
     lastPartitionId = reducePartitionId;
     if (outputTempFile == null) {
       outputTempFile = Utils.tempFileWith(outputFile);
+      // SPARK-37618: Create the file as group writable so it can be deleted 
by the shuffle service
+      Utils.createFileAsGroupWritable(outputTempFile);

Review comment:
       Write permission on the hosting directory(s) should be sufficient to 
delete - not on file itself.

##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -311,16 +313,49 @@ 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.foreach { mapStatus =>
+        // Port should always be external shuffle port if external shuffle is 
enabled
+        val isShufflePort = mapStatus.location.port == 
externalShuffleServicePort
+        val executorDeallocated = 
!blockManagerIdByExecutor.contains(mapStatus.location.executorId)
+        if (isShufflePort && executorDeallocated) {
+          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 =

Review comment:
       nit: `removeShuffleFromShuffleServicesFutures`

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2742,6 +2743,16 @@ private[spark] object Utils extends Logging {
     new File(path.getAbsolutePath + "." + UUID.randomUUID())
   }
 
+  /**
+   * Creates a file with group write permission.
+   */
+  def createFileAsGroupWritable(file: File): Unit = {
+    val perms = PosixFilePermissions.fromString("rw-rw----")
+    val path = file.toPath
+    Files.createFile(path)
+    Files.setPosixFilePermissions(path, perms)

Review comment:
       +CC @otterc There were some concerns with `PosixFilePermissions` which 
resulted in addition of `DiskBlockManager.createDirWithPermission770`
   Will this be affected by the same ?

##########
File path: 
core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala
##########
@@ -311,16 +313,49 @@ 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.foreach { mapStatus =>
+        // Port should always be external shuffle port if external shuffle is 
enabled
+        val isShufflePort = mapStatus.location.port == 
externalShuffleServicePort
+        val executorDeallocated = 
!blockManagerIdByExecutor.contains(mapStatus.location.executorId)
+        if (isShufflePort && executorDeallocated) {
+          val blocks = 
blocksToDeleteByShuffleService.getOrElseUpdate(mapStatus.location,
+            new mutable.HashSet[BlockId])
+          val blocksToDel =
+            shuffleManager.shuffleBlockResolver.getBlocksForShuffle(shuffleId, 
mapStatus.mapId)
+          blocksToDel.foreach(blocks.add(_))

Review comment:
       `blocks ++= blocksToDel`




-- 
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]

Reply via email to