You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/03/31 01:05:11 UTC

[GitHub] [spark] zhouyejoe opened a new pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

zhouyejoe opened a new pull request #32007:
URL: https://github.com/apache/spark/pull/32007


   What changes were proposed in this pull request?
   This is one of the patches for SPIP SPARK-30602 which is needed for push-based shuffle.
   Summary of changes:
   
   Executor will create the merge directories under the application temp directory provided by YARN. The access control of the folder will be set to 770, where Shuffle Service can create merged shuffle files and write merge shuffle data in to those files.
   Serve the merged shuffle blocks fetch request, read the merged shuffle blocks.
   
   Why are the changes needed?
   Refer to the SPIP in SPARK-30602.
   
   Does this PR introduce any user-facing change?
   No
   
   How was this patch tested?
   Added unit tests.
   The reference PR with the consolidated changes covering the complete implementation is also provided in SPARK-30602.
   We have already verified the functionality and the improved performance as documented in the SPIP doc.
   
   Lead-authored-by: Min Shen mshen@linkedin.com
   Co-authored-by: Chandni Singh chsingh@linkedin.com
   Co-authored-by: Ye Zhou yezhou@linkedin.com


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615527944



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       @zhouyejoe As I mentioned earlier, this edge case where merge_manager directory gets deleted, is handled by the fallback. The client will fetch original blocks instead of merged blocks. merge directory is not related to a block manager directory so why should it be created under it? If the goal is just to handle this edge case then this doesn't seem to be a clean solution.
   What is the benefit of making this change which is not clean when we can fallback for these exceptional cases?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       @zhouyejoe As I mentioned earlier, this edge case where merge_manager directory gets deleted, is handled by the fallback. The client will fetch original blocks instead of merged blocks. merge directory is not related to a block manager directory so why should it be created under it? If the goal is just to handle this edge case then this doesn't seem to be a clean solution.
   What is the benefit of making this change which is not clean when we can rely on fallback for these exceptional cases?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638035051



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt.
+          // Former attempt's shuffle partitions information will also be cleaned up.
+          boolean newAttemptRegistered = false;
+          if (appsPathsInfo.containsKey(appId)
+              && mergeDirectoryMeta.attemptId > appsPathsInfo.get(appId).attemptId) {

Review comment:
       nit: 2 indents.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615286521



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       The scenario I am thinking of is as follows:
   
   app1-atttempt1 fails - but due to some factor (overloaded RM, out-of-band messages, slow NM/ESS netty/IO, etc), ESS on nodes which are not running containers for app1-attempt1 are not immediately notified.
   app1-attempt2 starts and subsequently starts shuffling data.
   Assume very quick tasks/bootstrap/etc for simplification just for the scenario.
   
   In this case, can there be a problem ? ESS writes to a merge directory which is going to get deleted when app1-attempt1 failure is finally communicated to it ?
   
   
   +CC @tgravescs who has much more context about these things than I do :-)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-853810536


   I left a few comments, most are minor. It's a good move to handle issues separately.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644517373



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       Fixed and added a unit test in HostLocalShuffleReadingSuite to check whether the hostLocalDirManager gets initiated when push based shuffle is enabled.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642215418



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +198,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {

Review comment:
       @zhouyejoe The earlier comment must be because the PR didn't have latest code and it was needed for initializing `activeMergedShuffleDirs`. There is no need for `activeMergedShuffleDirs`. As mentioned in the other comment it is not being used anywhere in `DiskBlockManager`. The dirs are being passed by the methods so why does this need to return the files.
   cc @mridulm 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638425206



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       +CC @otterc - please do check if this handles the case you referenced to above.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-846476449


   > 1. `RemoteBlockPushResolver` needs to ignore any `PushBlock` message that is from previous attempts otherwise it will still merge a block of previous attempt to files of latest attempt and this is going to corrupt merged files.
   > 2. We should try to keep active partitions info in `partitions` map and delete stale entries (partition info belonging to old attempts).
   > 3. Need to add UTs for the server to ignore any pushblock messages from previous attempts.
   > 4. I don't think we need to add attemptId to FinalizeMerge message
   
   1. Added the part to ignore the PushBlock from previous attempt
   2. Added the part to delete stale entries in partitions hashmap
   3. Added UT
   4. As discussed above, it is still needed to have the attemptID in the FinalizeMerge message


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644969959



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (!dirs.exists(_.nonEmpty)) {
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627646535



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To add to my comment above (should have provided more context): I was assuming we are relying on directory existence to infer latest.
   But I like @Ngone51's idea here better:
   >  And if the executor becomes the one who creates the merge dir, we send the ExecutorShuffleInfo with the special shuffleManager, e.g., "sort_merge_manager_attemptX". And ExternalBlockHandler can parse the shuffleManager into two parts
   
   `shuffleManager` should to be relevant when we had multiple shuffle managers - and from evolution point of view, we could always introduce a new shuffle in future.
   My proposal is extension of the idea above - here, attempt is metadata about shuffleManager we want to convey.
   Ideally, this should be within `ExecutorShuffleInfo` - but given the compatibility issues here, why not add a general way to encode metadata about the shuffle manager here ?
   
   That is, instead of hardcoding 'SortShuffleManager_attemptX' and 'SortShuffleManager' as supported patterns, we could simply allow for 'SortShuffleManager' (existing) and 'SortShuffleManager:json_string' ?
   This can allow for any potential future evolution to also be possible - with currently merge_dir: "merge_directory_\<attemptid\>" being the initial field (@zhouyejoe let us explicitly pass the directory name, instead of inferring it from attempt id ?)
   
   Ofcourse if ':' is missing in shuffleManager, then we treat it as empty metadata case.
   
   With this in place, we would still need changes to push block protocol to include attempt id - but given no one is using that currently, we can make that change.
   With this explicit specification of attempt id by executor, we remove all ambiguity.
   
   Thoughts @Ngone51 ?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615286521



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       The scenario I am thinking of is as follows:
   
   app1-atttempt1 fails - but due to some factor (overloaded RM, out-of-band messages, slow NM/ESS netty/IO, etc), ESS on nodes which are not running containers for app1-attempt1 are not immediately notified.
   app1-attempt2 starts and subsequently starts shuffling data.
   Assume very quick tasks/bootstrap/etc for simplification just for the scenario.
   
   In this case, can there be a problem ? ESS writes to a merge directory which is going to get deleted when app1-attempt1 failure is finally communicated to it ? Any other complications/failure/inconsistency modes in such a case ?
   
   Or is this gauranteed not to happen ? (I dont believe so, but would like to understand better).
   
   
   +CC @tgravescs who has much more context about these things than I do :-)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r624054924



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       @Ngone51 Moving the management to executor is a nice idea - If we are moving the delete to the executor, then we could formulate it to make the change minimal right ?
   
   In executor: 
   a) List application/merge_manager_*
   b) Delete all merge_manager_* directories [1] which are older than the current attempt id [2]
   b.1) If there are newer merge_manager_* directories - it exit's : since this is not the latest attempt for this app (should be vanishingly small chance - but adding for completeness sake).
   c) Create merge_manager_$ATTEMPT_ID, and register with existing rpc's.
   
   In shuffle service:
   d) Shuffle service simply looks up the latest merge_manager_* (which should ideally be a single directory), and registers application/executor with that attempt.
   d.1) If earlier attempts exist, cleanup metadata - directory would be cleaned up already anyway.
   
   Thoughts ?
   
   [1] Given possibility of concurrent executors deleting, we should be robust to the errors there.
   [2] In theory, there could be race with a newer version - but should not happen.
    - that is, if merge_manager_attemptId




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859105499


   The spark UI test which failed in jenkins is unrelated to this PR.
   Merging to master.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r622429509



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Below are my comments for each solution 
   1. Create a new `RegisterExecutor` message. If we do this, we might also want to add the recent "merge_manager_attemptx` name. A benefit of that would be to remove the logic on the server which tries to find the merge_manager directory path by finding the parent of blockMgrs directory.
   
   2. Encoding `attemptId` in the `appId` of `RegisterExecutor` message. The old `ExternalShuffleBlockResolver` needs to know the `appId` (without attemptId) when the executors are registered and the `executors` map is populated, since all the `get...BlockData()` apis reference this map with only `appId` and `executorId`. We can't change these apis. `RemoteBlockPushResolver` may also need to know the `appId` and `attemptId` separately to avoid changes to multiple data-structures. It would be then better to handle this parsing in `ExternalBlockHandler`. 
   
   3. Have the `RemoteBlockPushResolver` figure out the attemptId just by itself by listing the directory and finding the latest attempt. With this one every time an executor registers, the shuffle server has to do the following:
   - find the parent directory from blockManager dirs. 
   - list the merge_manager_* dirs  
   - figure out the latest attempt.
   All these 3 steps above are avoided in solution 2 for every executor registration because we know the attempt Id from the message itself and it will ignore the ones belonging to the current one. The additional cost that is incurred in solution 2 is just splitting the `appId` and `attemptId` which I don't think is that much.
   
   I think Solution 1 is the cleanest but I can see that it adds another message and attemptId may be just specific to Yarn. Between 2 and 3, I prefer solution 2.
   
   WDYT @tgravescs @mridulm @attilapiros @Ngone51 @Victsm @zhouyejoe 
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638023754



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -529,10 +530,17 @@ private[spark] class BlockManager(
 
   private def registerWithExternalShuffleServer(): Unit = {
     logInfo("Registering executor with local external shuffle service.")
+    val shuffleManagerMeta =
+      if (conf.get(config.PUSH_BASED_SHUFFLE_ENABLED)) {
+        s"${shuffleManager.getClass.getName}:" +
+          s"${diskBlockManager.getMergeDirectoryAndAttemptIDJsonString()}}}"

Review comment:
       BTW, shall we only add metadata for YARN only?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615286745



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       Ah, umask ... interesting. Not sure why it does not impact mkdir then ? Thoughts ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637978835



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +105,35 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(java.lang.String,
+   *   java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String): File = {
+    if (activeMergedShuffleDirs.isEmpty) {
+      throw new IllegalArgumentException(
+        s"Cannot read $filename because active merged shuffle dirs is empty")
+    }
+    val localDirsForMergedShuffleBlock = activeMergedShuffleDirs.map(_.getPath)
+    ExecutorDiskUtils.getFile(localDirsForMergedShuffleBlock, subDirsPerLocalDir, filename)
+  }
+
+

Review comment:
       nit: redundant blank line.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637965145



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +59,21 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  private val mergeDirName: String = {
+    try {
+      val attemptId = conf.get("spark.app.attempt.id")
+      MERGE_DIRECTORY + "_" + Utils.sanitizeDirName(attemptId)

Review comment:
       IIUC, `attemptId` is only an integer. Why do we need to sanitize it?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615285997



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))

Review comment:
       No, pls dont change !
   I was adding it as a reference for reviewers for context :)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627646535



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To add to my comment above (should have provided more context): I was assuming we are relying on directory existence to infer latest.
   But I like @Ngone51's idea here better:
   >  And if the executor becomes the one who creates the merge dir, we send the ExecutorShuffleInfo with the special shuffleManager, e.g., "sort_merge_manager_attemptX". And ExternalBlockHandler can parse the shuffleManager into two parts
   
   `shuffleManager` should to be relevant when we had multiple shuffle managers - and from evolution point of view, we could always introduce a new shuffle in future.
   My proposal is extension of the idea above - here, attempt is metadata about shuffleManager we want to convey.
   Ideally, this should be within `ExecutorShuffleInfo` - but given the compatibility issues here, why not add a general way to encode metadata about the shuffle manager here ?
   
   That is, instead of hardcoding 'SortShuffleManager_attemptX' and 'SortShuffleManager' as supported patterns, we could simply allow for 'SortShuffleManager' (existing) and 'SortShuffleManager:json_string' ?
   This can allow for any potential future evolution to also be possible - with currently `{"merge_dir": "merge_directory_\<attemptid\>", "attempt_id":\<attempt_id\>}` being the initial field (@zhouyejoe let us explicitly pass the directory name, instead of inferring it from attempt id ?)
   
   Ofcourse if ':' is missing in shuffleManager, then we treat it as empty metadata case.
   
   With this in place, we would still need changes to push block protocol to include attempt id - but given no one is using that currently, we can make that change.
   With this explicit specification of merge directory, we remove all ambiguity.
   
   Thoughts @Ngone51 ?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r611987404



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +728,24 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId)
+  }
+
+  /**
+   * Get the local merged shuffle block metada data for the given block ID.
+   */
+  def getMergedBlockMeta(blockId: ShuffleBlockId): MergedBlockMeta = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId)
+  }
+
+

Review comment:
       @zhouyejoe This is missing a change where `hostLocalDirManager` needs to initialized at line 505 when push based shuffle is enabled. Like this:
   ```
       hostLocalDirManager = {
         // PART OF SPARK-33350
         if ((conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
           !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL))
             ||  Utils.isPushBasedShuffleEnabled(conf)) {
           Some(new HostLocalDirManager(
             futureExecutionContext,
             conf.get(config.STORAGE_LOCAL_DISK_BY_EXECUTORS_CACHE_SIZE),
             blockStoreClient))
         } else {
           None
         }
       }
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644423043



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +349,64 @@ void deleteExecutorDirs(Path[] dirs) {
     }
   }
 
+  /**
+   * Create StreamCallback for invalid push blocks with the specific error message.
+   * If specific error message is null, this StreamCallback won't throw exception in client.
+   */
+  private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+      String streamId,
+      String errorMessage) {
+    return new StreamCallbackWithID() {
+      @Override
+      public String getID() {
+        return streamId;
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) {
+        // Ignore the requests. It reaches here either when a request is received after the
+        // shuffle file is finalized or when a request is for a duplicate block.
+      }
+
+      @Override
+      public void onComplete(String streamId) {
+        if (errorMessage != null) {
+          // Throw an exception here so the block data is drained from channel and server
+          // responds RpcFailure to the client.
+          throw new RuntimeException(String.format("Block %s %s", streamId, errorMessage));
+        }
+        // For duplicate block that is received before the shuffle merge finalizes, the
+        // server should respond success to the client.
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable cause) {
+      }
+    };
+  }
+
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppAttemptPathsInfo appAttemptPathsInfo = getAppAttemptPathsInfo(msg.appId);
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId =
+      new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+      // If this Block belongs to a former application attempt, it is considered late,
+      // as only the blocks from the current application attempt will be merged
+      // TODO: [SPARK-35548] Client should be updated to handle this error.
+      return createCallbackForInvalidPushBlocks(streamId,

Review comment:
       Removed from the slim down PR.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638034589



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt.
+          // Former attempt's shuffle partitions information will also be cleaned up.

Review comment:
       nit: "...attempt, and former..."




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r621391061



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This is a good point, unfortunately we do not set the attempt id in spark conf like we do for app id.
   A few options here would be:
   
   a) Also propagate attempt id via "spark.app.attemptId" when available (and use default value if missing).
   b) Defer registeration/directory creation to first task being run : ShuffleMapTask has `appAttemptId` as part of it. 
   c) For yarn, CONTAINER_ID env variable can be parsed to fetch attempt id - though this might not be optimal.
   
   I am not very keen on modifying protocol if possible.
   
   Thoughts ?
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614521383



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       @mridulm I've tried this last year and it doesn't work. It still creates the directory with permission `750`.  Internally we have this  TODO as well for this method.
   ```
      * TODO: Find out why can't we create a dir using java api with permission 770
      *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
      *  PosixFilePermissions.fromString("rwxrwx---")))
      ```
   This has something to do yarn setting a umask when it starts the container process. I don't remember the details because I tried this early last year.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645326016



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       Moved to DiskBlockManager. Also need to change the access modifiers for MAX_DIR_CREATION_ATTEMPTS in Utils.scala.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r647919473



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +729,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def getLocalMergedBlockMeta(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): MergedBlockMeta = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId, Some(dirs))

Review comment:
       Added unit tests in IndexShuffleBlockResolver to test the getMergedBlockData and getMergedBlockMeta




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648875063



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,64 +123,85 @@ public ShuffleIndexInformation load(File file) throws IOException {
    * shuffle does not exist, initializes the metadata.
    */
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
-      AppShuffleId appShuffleId,
+      AppAttemptShuffleId appAttemptShuffleId,
+      AppAttemptPathsInfo appAttemptPathsInfo,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
-    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+    File dataFile = getMergedShuffleDataFile(
+      appAttemptPathsInfo, appAttemptShuffleId.appId, appAttemptShuffleId.shuffleId, reduceId);
+    if (!partitions.containsKey(appAttemptShuffleId) && dataFile.exists()) {
       // If this partition is already finalized then the partitions map will not contain
       // the appShuffleId but the data file would exist. In that case the block is considered late.
       return null;
     }
-    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
-      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
-    return shufflePartitions.computeIfAbsent(reduceId, key -> {
-      // It only gets here when the key is not present in the map. This could either
-      // be the first time the merge manager receives a pushed block for a given application
-      // shuffle partition, or after the merged shuffle file is finalized. We handle these
-      // two cases accordingly by checking if the file already exists.
-      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
-      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
-      try {
-        if (dataFile.exists()) {
-          return null;
-        } else {
-          return newAppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
-        }
-      } catch (IOException e) {
-        logger.error(
-          "Cannot create merged shuffle partition with data file {}, index file {}, and "
-            + "meta file {}", dataFile.getAbsolutePath(),
-            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
-        throw new RuntimeException(
-          String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s "
-          + "reduceId %s", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+    // While processing the application remove, where the shuffle partitions info for the specific
+    // application will be cleaned up, this method will still be called to create new partitions
+    // as of receiving the push blocks. To avoid the potential memory leak, before creating the
+    // empty hashmap for storing the shuffle partitions information in the partitions hashmap,
+    // we need to make sure that the entry for the specific application must still exist in
+    // appAttemptsPathInfo hashmap. Otherwise, the push blocks should be ignored.

Review comment:
       Follow up on SPARK-35546




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe edited a comment on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe edited a comment on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-850036241


   Created ticket for later improvement [SPARK-35546](https://issues.apache.org/jira/browse/SPARK-35546)


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640996792



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -336,51 +436,26 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
       && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
         : partitionInfoBeforeCheck;
-    final String streamId = String.format("%s_%d_%d_%d",
-      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
-      msg.reduceId);
     if (partitionInfo != null) {
       return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
     } else {
       // For a duplicate block or a block which is late, respond back with a callback that handles
       // them differently.
-      return new StreamCallbackWithID() {
-        @Override
-        public String getID() {
-          return streamId;
-        }
-
-        @Override
-        public void onData(String streamId, ByteBuffer buf) {
-          // Ignore the requests. It reaches here either when a request is received after the
-          // shuffle file is finalized or when a request is for a duplicate block.
-        }
-
-        @Override
-        public void onComplete(String streamId) {
-          if (isTooLate) {
-            // Throw an exception here so the block data is drained from channel and server
-            // responds RpcFailure to the client.
-            throw new RuntimeException(String.format("Block %s %s", streamId,
-              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
-          }
-          // For duplicate block that is received before the shuffle merge finalizes, the
-          // server should respond success to the client.
-        }
-
-        @Override
-        public void onFailure(String streamId, Throwable cause) {
-        }
-      };
+      if (isTooLate) {

Review comment:
       The implementation of the createCallbackForInvalidPushBlocks will have a check for the error message. If the error message is null, it won't throw RuntimeException. Will add a comment here. Refactoring this part is for different types of error message when creating the callback.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-850036241


   Created ticket for later improvement https://issues.apache.org/jira/browse/SPARK-35546


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638416776



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -19,8 +19,6 @@
 
 import com.google.common.base.Objects;
 import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;

Review comment:
       Rebase your dev branch so to not revert these more recent changes.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638114426



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -116,29 +115,35 @@ public ShuffleIndexInformation load(File file) throws IOException {
    * application, retrieves the associated metadata. If not present and the corresponding merged
    * shuffle does not exist, initializes the metadata.
    */
-  private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
-      AppShuffleId appShuffleId,
+  private AppAttemptShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
+      AppAttemptShuffleId appAttemptShuffleId,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
-    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+    if (!appsPathsInfo.containsKey(appAttemptShuffleId.appId)

Review comment:
       We are checking `appPathsInfo` for the attempt so this can be pulled out of here and checked before calling this method

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -761,28 +818,31 @@ public boolean equals(Object o) {
       if (o == null || getClass() != o.getClass()) {
         return false;
       }
-      AppShuffleId that = (AppShuffleId) o;
-      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+      AppAttemptShuffleId that = (AppAttemptShuffleId) o;
+      return Objects.equal(appId, that.appId)
+        && attemptId == that.attemptId
+        && shuffleId == that.shuffleId;
     }
 
     @Override
     public int hashCode() {
-      return Objects.hashCode(appId, shuffleId);
+      return Objects.hashCode(appId, attemptId, shuffleId);
     }
 
     @Override
     public String toString() {
-      return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
-        .append("appId", appId)
-        .append("shuffleId", shuffleId)
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("attemptId", attemptId)
+        .add("shuffleId", shuffleId)
         .toString();
     }
   }
 
   /** Metadata tracked for an actively merged shuffle partition */
-  public static class AppShufflePartitionInfo {
+  public static class AppAttemptShufflePartitionInfo {

Review comment:
       We only create a partition info for the latest attempt. I don't really think renaming this class is necessary. 
   The java doc also says this. Here `attempt` has no significance since it is just the part of the `AppAttemptShuffleId` which is renamed.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+    appId: String,
+    shuffleId: Int,
+    reduceId: Int) extends BlockId {
+  override def name: String =
+    "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".index"
+}
+
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedMetaBlockId(
+    appId: String,
+    shuffleId: Int,
+    reduceId: Int) extends BlockId {
+  override def name: String =
+    "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".meta"

Review comment:
       So the file names that the client expects now starts with `shuffleMerged`. Has this be changed on the server side which writes these files? IIRC it was writing to files starting with `mergedShuffle`

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.

Review comment:
       Are there UTs added for these cases?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ErrorHandler.java
##########
@@ -59,7 +59,7 @@ default boolean shouldLogError(Throwable t) {
      * will not retry pushing the block nor log the exception on the client side.
      */
     public static final String TOO_LATE_MESSAGE_SUFFIX =
-      "received after merged shuffle is finalized";
+      "received after merged shuffle is finalized or newer attempt has started";

Review comment:
       Why don't we make "newer attempt has started" a separate message?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -92,7 +91,7 @@
   public RemoteBlockPushResolver(TransportConf conf) {
     this.conf = conf;
     this.partitions = Maps.newConcurrentMap();
-    this.appsPathInfo = Maps.newConcurrentMap();
+    this.appsPathsInfo = Maps.newConcurrentMap();

Review comment:
       Nit: rename to appAttemptPathsInfo

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -293,9 +307,9 @@ void deleteExecutorDirs(Path[] dirs) {
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
-    AppShufflePartitionInfo partitionInfoBeforeCheck =
-      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId = new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    AppAttemptShufflePartitionInfo partitionInfoBeforeCheck =

Review comment:
       We can just check before creating the partitionInfo, whether the attemptId is latest or not. If not, then throw the relevant exception here. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644834562



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       @Ngone51 @zhouyejoe How about moving this function to DiskBlockManager completely? For more restrictive permissions than `750` we can  use java api. Here, we are trying to run the command `mkdir` just because we want to create mergeDirectory with `770` permission. 

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&

Review comment:
       Nit: is the String "yarn" declared as a constant anywhere else that we can use?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       This still looks the same @zhouyejoe. `hostLocalDirManager` should be set when push-based shuffle is enable. Can you put `()` around  `conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)` so that it is clear.
   It should be:
   ```
   if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
             Utils.isPushBasedShuffleEnabled(conf))
    ```

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {

Review comment:
       nit: Change it to `ShuffleMergedDataBlockId`

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // This executor does not find merge_manager directory, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.

Review comment:
       This comment needs to be updated as well. We are no longer working with this assumption.
   `but we are working on the assumption that the executors launched around the same time
             // will have the same set of application local directories.`
    Also move it inside the `if` block below.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       ok

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -139,7 +171,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
    * located inside configured local directories and won't
    * be deleted on JVM exit when using the external shuffle service.
    */
-  private def createLocalDirs(conf: SparkConf): Array[File] = {
+  private def createLocalDirs(): Array[File] = {

Review comment:
       Nit: is this change really required? This is original code so why not just leave it as it is?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -40,7 +42,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   /* Create one local directory for each path mentioned in spark.local.dir; then, inside this
    * directory, create multiple subdirectories that we will hash files into, in order to avoid
    * having really large inodes at the top level. */
-  private[spark] val localDirs: Array[File] = createLocalDirs(conf)
+  private[spark] val localDirs: Array[File] = createLocalDirs()

Review comment:
       Nit: this change seems unrelated to push-based shuffle so should we not make it?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614554227



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Sorry, I thought this comment was about adding stage attempt Id. However, this is about application attempt Id which is not covered by SPARK-32923. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648261185



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"merge_manager dir does not exist")

Review comment:
       nit: use `MERGE_MANAGER_DIR`

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedDataBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+    appId: String,
+    shuffleId: Int,
+    reduceId: Int) extends BlockId {
+  override def name: String =
+    "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Shall we reuse `RemoteBlockPushResolver.MERGED_SHUFFLE_FILE_NAME_PREFIX`?
   
   (same for `ShuffleMergedMetaBlockId`, `ShuffleMergedDataBlockId`)

##########
File path: core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
##########
@@ -161,4 +166,78 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa
     val resolver = new IndexShuffleBlockResolver(conf, blockManager)
     assert(resolver.getMigrationBlocks(ShuffleBlockInfo(Int.MaxValue, Long.MaxValue)).isEmpty)
   }
+
+  test("getMergedBlockData should return expected FileSegmentManagedBuffer list") {
+    val shuffleId = 1
+    val reduceId = 1
+    val dataFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.data"
+    val dataFile = new File(tempDir.getAbsolutePath, dataFileName)
+    val out = new FileOutputStream(dataFile)
+    Utils.tryWithSafeFinally {
+      out.write(new Array[Byte](30))
+    } {
+      out.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val managedBufferList =
+      resolver.getMergedBlockData(ShuffleBlockId(shuffleId, -1, reduceId), dirs)

Review comment:
       nit: use `SHUFFLE_PUSH_MAP_ID` instead of `-1`?

##########
File path: core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
##########
@@ -161,4 +166,78 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa
     val resolver = new IndexShuffleBlockResolver(conf, blockManager)
     assert(resolver.getMigrationBlocks(ShuffleBlockInfo(Int.MaxValue, Long.MaxValue)).isEmpty)
   }
+
+  test("getMergedBlockData should return expected FileSegmentManagedBuffer list") {
+    val shuffleId = 1
+    val reduceId = 1
+    val dataFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.data"
+    val dataFile = new File(tempDir.getAbsolutePath, dataFileName)
+    val out = new FileOutputStream(dataFile)
+    Utils.tryWithSafeFinally {
+      out.write(new Array[Byte](30))
+    } {
+      out.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val managedBufferList =
+      resolver.getMergedBlockData(ShuffleBlockId(shuffleId, -1, reduceId), dirs)
+    assert(managedBufferList.size === 3)
+    assert(managedBufferList(0).size === 10)
+    assert(managedBufferList(1).size === 0)
+    assert(managedBufferList(2).size === 20)
+  }
+
+  test("getMergedBlockMeta should return expected MergedBlockMeta") {
+    val shuffleId = 1
+    val reduceId = 1
+    val metaFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.meta"
+    val metaFile = new File(tempDir.getAbsolutePath, metaFileName)
+    val chunkTracker = new RoaringBitmap()
+    chunkTracker.add(1)
+    chunkTracker.add(2)
+    val metaFileOutputStream = new FileOutputStream(metaFile)
+    val outMeta = new DataOutputStream(metaFileOutputStream)
+    Utils.tryWithSafeFinally {
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(3)
+      chunkTracker.add(4)
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(5)
+      chunkTracker.add(6)
+      chunkTracker.serialize(outMeta)
+    }{
+      outMeta.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val mergedBlockMeta =
+      resolver.getMergedBlockMeta(ShuffleBlockId(shuffleId, -1, reduceId), dirs)
+    assert(mergedBlockMeta.getNumChunks === 3)
+    assert(mergedBlockMeta.readChunkBitmaps().size === 3)
+    assert(mergedBlockMeta.readChunkBitmaps()(0).contains(1))
+    assert(mergedBlockMeta.readChunkBitmaps()(0).contains(2))

Review comment:
       Shall we check other two chunks?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"merge_manager dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)

Review comment:
       nit: "merge" -> "merge_manager"

##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleReadingSuite.scala
##########
@@ -133,4 +133,13 @@ class HostLocalShuffleReadingSuite extends SparkFunSuite with Matchers with Loca
       assert(remoteBytesRead.sum === 0 && remoteBlocksFetched.sum === 0)
     }
   }
+
+  test("Enable host local shuffle reading when Push based shuffle is enabled") {

Review comment:
       nit: "Push" -> "push"

##########
File path: core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
##########
@@ -161,4 +166,78 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa
     val resolver = new IndexShuffleBlockResolver(conf, blockManager)
     assert(resolver.getMigrationBlocks(ShuffleBlockInfo(Int.MaxValue, Long.MaxValue)).isEmpty)
   }
+
+  test("getMergedBlockData should return expected FileSegmentManagedBuffer list") {
+    val shuffleId = 1
+    val reduceId = 1
+    val dataFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.data"
+    val dataFile = new File(tempDir.getAbsolutePath, dataFileName)
+    val out = new FileOutputStream(dataFile)
+    Utils.tryWithSafeFinally {
+      out.write(new Array[Byte](30))
+    } {
+      out.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val managedBufferList =
+      resolver.getMergedBlockData(ShuffleBlockId(shuffleId, -1, reduceId), dirs)
+    assert(managedBufferList.size === 3)
+    assert(managedBufferList(0).size === 10)
+    assert(managedBufferList(1).size === 0)
+    assert(managedBufferList(2).size === 20)
+  }
+
+  test("getMergedBlockMeta should return expected MergedBlockMeta") {
+    val shuffleId = 1
+    val reduceId = 1
+    val metaFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.meta"
+    val metaFile = new File(tempDir.getAbsolutePath, metaFileName)
+    val chunkTracker = new RoaringBitmap()
+    chunkTracker.add(1)
+    chunkTracker.add(2)
+    val metaFileOutputStream = new FileOutputStream(metaFile)
+    val outMeta = new DataOutputStream(metaFileOutputStream)
+    Utils.tryWithSafeFinally {
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(3)
+      chunkTracker.add(4)
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(5)
+      chunkTracker.add(6)
+      chunkTracker.serialize(outMeta)
+    }{
+      outMeta.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val mergedBlockMeta =
+      resolver.getMergedBlockMeta(ShuffleBlockId(shuffleId, -1, reduceId), dirs)

Review comment:
       ditto




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645321971



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       Updated as @mridulm suggested.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648743073



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -744,10 +771,12 @@ AppShufflePartitionInfo getPartitionInfo() {
    */
   public static class AppShuffleId {
     public final String appId;
+    public final int attemptId;
     public final int shuffleId;
 
-    AppShuffleId(String appId, int shuffleId) {
+    AppShuffleId(String appId, int attemptId, int shuffleId) {

Review comment:
       Will resolve the multi-attempts in SPARK-35546




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644969959



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (dirs.exists(_.nonEmpty)) {
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644834562



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       @Ngone51 @zhouyejoe How about moving this function to DiskBlockManager completely? For more restrictive permissions than `750` we can  use java api. Here, we are trying to run the command `mkdir` just because we want to create mergeDirectory with `770` permission. 

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&

Review comment:
       Nit: is the String "yarn" declared as a constant anywhere else that we can use?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       This still looks the same @zhouyejoe. `hostLocalDirManager` should be set when push-based shuffle is enable. Can you put `()` around  `conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)` so that it is clear.
   It should be:
   ```
   if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
             Utils.isPushBasedShuffleEnabled(conf))
    ```

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {

Review comment:
       nit: Change it to `ShuffleMergedDataBlockId`

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // This executor does not find merge_manager directory, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.

Review comment:
       This comment needs to be updated as well. We are no longer working with this assumption.
   `but we are working on the assumption that the executors launched around the same time
             // will have the same set of application local directories.`
    Also move it inside the `if` block below.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       ok

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -139,7 +171,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
    * located inside configured local directories and won't
    * be deleted on JVM exit when using the external shuffle service.
    */
-  private def createLocalDirs(conf: SparkConf): Array[File] = {
+  private def createLocalDirs(): Array[File] = {

Review comment:
       Nit: is this change really required? This is original code so why not just leave it as it is?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -40,7 +42,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   /* Create one local directory for each path mentioned in spark.local.dir; then, inside this
    * directory, create multiple subdirectories that we will hash files into, in order to avoid
    * having really large inodes at the top level. */
-  private[spark] val localDirs: Array[File] = createLocalDirs(conf)
+  private[spark] val localDirs: Array[File] = createLocalDirs()

Review comment:
       Nit: this change seems unrelated to push-based shuffle so should we not make it?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       Yeah you are right Mridul. It should be 
   ```
   if ((conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) ||
             Utils.isPushBasedShuffleEnabled(conf))
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r622429509



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Below are my comments for each solution 
   1. Create a new `RegisterExecutor` message. If we do this, we might also want to add the recent "merge_manager_attemptx` name. A benefit of that would be to remove the logic on the server which tries to find the merge_manager directory path by finding the parent of blockMgrs directory.
   
   2. Encoding `attemptId` in the `appId` of `RegisterExecutor` message. The old `ExternalShuffleBlockResolver` needs to know the `appId` (without attemptId) when the executors are registered and the `executors` map is populated, since all the `get...BlockData()` apis reference this map with only `appId` and `executorId`. We can't change these apis. `RemoteBlockPushResolver` may also need to know the `appId` and `attemptId` separately to avoid changes to multiple data-structures. It would be then better to handle this parsing in `ExternalBlockHandler`. 
   
   3. Have the `RemoteBlockPushResolver` figure out the attemptId just by itself by listing the directory and finding the latest attempt. With this one every time an executor registers, the remote shuffle has to do the following:
   - find the parent directory from blockManager dirs. 
   - list the merge_manager_* dirs  
   - figure out the latest attempt.
   All these 3 steps above are avoided in solution 2 for every executor registration because we know the attempt Id from the message itself and it will ignore the ones belonging to the current one. The additional cost that is incurred in solution 2 is just splitting the `appId` and `attemptId` which I don't think is that much.
   
   I think Solution 1 is the cleanest but I can see that it adds another message and attemptId may be just specific to Yarn. Between 2 and 3, I prefer solution 2.
   
   WDYT @tgravescs @mridulm @attilapiros @Ngone51 @Victsm @zhouyejoe 
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627110838



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Thanks for all the discussion above. I will update the PR with the proposed ideas above: 1. No new RPC needs to be introduced. 2. The dirs will still be created by Executor as of the the permission issue. 3. Will let the Executor manage the merge_dir creation. 4. Will let the Executor to delete the last attempt's created merge_dir if it exists. 5. Register the merge attempt Dirs through ExecutorShuffleInfo, where a new String for shuffleManager field to distinguish the attemptID, e.g., "sort_merge_manager_attemptX", suggested by @Ngone51 . 
   
   @mridulm  IIUC, if we have the item 5 there, we don't need the shuffle service to list the dirs and figure out the largest attemptIDs for merge dirs, right? 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637976278



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -190,3 +295,9 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 }
+
+private[spark] object DiskBlockManager {
+  private[spark] val MERGE_DIRECTORY = "merge_directory"
+  private[spark] val MERGE_DIR_KEY = "mergeDir"
+  private[spark] val ATTEMPT_ID_KEY = "attemptId"

Review comment:
       Do we have a helper class or object for push-based shuffle? It would be better to put these fields there.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-853615497


   Updated with a slim version, which excludes the handling for multiple attempts case. 
   @Ngone51  Would like to share a little bit more context. We had multiple round of discussion internally regarding this PR, and the agreement we have reached internally is to exclude the multiple attempts support from this PR, whereas we have created a ticket SPARK-30602 to add it later on.
   
   cc @mridulm @Victsm @otterc


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637964223



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +59,21 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  private val mergeDirName: String = {
+    try {
+      val attemptId = conf.get("spark.app.attempt.id")

Review comment:
       nit: `conf.getOption("spark.app.attempt.id").map(id => MERGE_DIRECTORY + "_" + id).getOrElse(MERGE_DIRECTORY)`




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859045755


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/139658/
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-810691560


   Can one of the admins verify this patch?


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858883870


   **[Test build #139658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139658/testReport)** for PR 32007 at commit [`e630725`](https://github.com/apache/spark/commit/e630725ca5c161cea62a2afcc7668a67a3e6d72e).


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615286521



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       The scenario I am thinking of is as follows:
   
   app1-atttempt1 fails - but due to some factor (overloaded RM, out-of-band messages, etc), ESS on nodes which are not running containers for app1-attempt1 are not immediately notified.
   app1-attempt2 starts and subsequently starts shuffling data.
   Assume very quick tasks/bootstrap/etc for simplification just for the scenario.
   
   In this case, can there be a problem ? ESS writes to a merge directory which is going to get deleted when app1-attempt1 failure is finally communicated to it ?
   
   
   +CC @tgravescs who has much more context about these things than I do :-)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r607501885



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt

Review comment:
       This assumption about length is fine as we are directly reading content into a `byte[]` and the number of chunks is reasonably bounded << `Int.MaxValue`.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       `appId` would not be sufficient - we can have multiple attempts for an application.
   (here and in other block id's introduced).

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))

Review comment:
       Minor note: `Files.newInputStream` has had issue in past (see SPARK-21475, specifically revert by Shixiong Zhu).
   While not relevant to this change specifically, adding for context to reviewers.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return
+        }
+      }
+      // Since this executor didn't see any merge_manager directories, it will start creating them.
+      // It's possible that the other executors launched at the same time may also reach here but
+      // we are working on the assumption that the executors launched around the same time will
+      // have the same set of application local directories.
+      localDirs.flatMap { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // Only one container will create this directory. The filesystem will handle any race
+          // conditions.
+          if (!mergeDir.exists()) {
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))
+              Utils.createDirWith770(sudDir)
+            }
+          }
+          logInfo(s"Merge directory at $mergeDir")
+          Some(mergeDir)
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+            None
+        }

Review comment:
       Some and None are not required after change to `foreach`

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return
+        }
+      }
+      // Since this executor didn't see any merge_manager directories, it will start creating them.
+      // It's possible that the other executors launched at the same time may also reach here but
+      // we are working on the assumption that the executors launched around the same time will
+      // have the same set of application local directories.
+      localDirs.flatMap { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // Only one container will create this directory. The filesystem will handle any race
+          // conditions.
+          if (!mergeDir.exists()) {
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))
+              Utils.createDirWith770(sudDir)
+            }
+          }
+          logInfo(s"Merge directory at $mergeDir")
+          Some(mergeDir)
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+            None
+        }
+      }
+      Utils.getConfiguredLocalDirs(conf).map(rootDir => new File(rootDir, MERGE_MANAGER_DIR))
+    }
+  }
+
+  private def findActiveMergedShuffleDirs(conf: SparkConf): Option[Array[File]] = {
+    Option(Utils.getConfiguredLocalDirs(conf).map(
+      rootDir => new File(rootDir, "merge_manager")).filter(mergeDir => mergeDir.exists()))

Review comment:
       `"merge_manager"` -> `MERGE_MANAGER_DIR`

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +53,14 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  createLocalDirsForMergedShuffleBlocks(conf)
+
+  private[spark] lazy val activeMergedShuffleDirs: Option[Array[File]] =
+    findActiveMergedShuffleDirs(conf)
+

Review comment:
       Why not populate this as result of `createLocalDirsForMergedShuffleBlocks` ?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return
+        }
+      }
+      // Since this executor didn't see any merge_manager directories, it will start creating them.
+      // It's possible that the other executors launched at the same time may also reach here but
+      // we are working on the assumption that the executors launched around the same time will
+      // have the same set of application local directories.
+      localDirs.flatMap { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // Only one container will create this directory. The filesystem will handle any race
+          // conditions.
+          if (!mergeDir.exists()) {
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))
+              Utils.createDirWith770(sudDir)
+            }
+          }
+          logInfo(s"Merge directory at $mergeDir")
+          Some(mergeDir)
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+            None
+        }
+      }
+      Utils.getConfiguredLocalDirs(conf).map(rootDir => new File(rootDir, MERGE_MANAGER_DIR))

Review comment:
       Dead code ?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +92,37 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.appId, mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.appId, mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.appId, mergedMetaBlockId.name)
+      case _ =>
+        throw new RuntimeException(s"Only merged block ID is supported, but got ${blockId}")
+    }
+  }
+
+  private def getMergedShuffleFile(appId: String, filename: String): File = {
+    if (activeMergedShuffleDirs.isEmpty) {
+      throw new RuntimeException(

Review comment:
       `RuntimeException` -> `IllegalStateException` ?
   

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +92,37 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.appId, mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.appId, mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.appId, mergedMetaBlockId.name)
+      case _ =>
+        throw new RuntimeException(s"Only merged block ID is supported, but got ${blockId}")

Review comment:
       `RuntimeException` -> `IllegalArgumentException` ?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return
+        }
+      }
+      // Since this executor didn't see any merge_manager directories, it will start creating them.
+      // It's possible that the other executors launched at the same time may also reach here but
+      // we are working on the assumption that the executors launched around the same time will
+      // have the same set of application local directories.
+      localDirs.flatMap { rootDir =>

Review comment:
       `flatMap` -> `foreach` ?
   Note, this and the comment below are for the current method - when `createLocalDirsForMergedShuffleBlocks` is changed to return active merge managers, some of these code paths could become relevant again.

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       Instead of invoking external program (`mkdir`), `PosixFilePermissions` wont work here ?
   Something like:
   
   ```
   val groupWritableAttribute = PosixFilePermissions.asFileAttribute(PosixFilePermissions.fromString("rwxrwx---"))
   val createdPath = Files.createDirectory(dirToCreate.getAbsolutePath, groupWritableAttribute)
   ```

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       Use `tryWithResource` instead.
   Something like
   ```
   val offsets = Utils.tryWithResource {
     new DataInputStream(Files.newInputStream(indexFile.toPath))
   } { dis =>
     val buffer = ByteBuffer.allocate(size)
     dis.readFully(buffer.array)
     buffer.asLongBuffer
   }
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614554227



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Sorry, I thought this comment was about adding stage attempt Id. However, this is about application attempt Id which is not covered by SPARK-32923. 
   Application local dirs are removed when the app fails so should not be an issue.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858977861


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/44186/
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638399675



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -419,4 +419,8 @@ public long mergedIndexCacheSize() {
   public int ioExceptionsThresholdDuringMerge() {
     return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
   }
+
+  public int appAttemptId() {
+    return conf.getInt("spark.app.attempt.id", -1);

Review comment:
       @Ngone51 Curious if you mean about the assumption of parsing attempt as an `int` ? Or simply about this (attempts) being used only in yarn currently ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638399675



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -419,4 +419,8 @@ public long mergedIndexCacheSize() {
   public int ioExceptionsThresholdDuringMerge() {
     return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
   }
+
+  public int appAttemptId() {
+    return conf.getInt("spark.app.attempt.id", -1);

Review comment:
       @Ngone51 Curious if you mean about the assumption of parsing attempt as an `int` ? Or simply about this being used only in yarn ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615456879



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This is not going to be quite simple . If we attach UUID.randomUUID to merge dir path, then there are 2 major issues with that:
   1. We want a single merge directory under an application location dir. When a new executor starts, it checks whether the merge_manager exists. It only creates it if the merge_manager doesn't exist. If there is a random UUID attached, then how to let the new executors know when **not** to create the new directory? For blockMgr dirs this is not a requirement and each executor creates its own directory.
   
   2. How does the server know the exact name of merge_manager directory which has a random UUID? The server currently constructs out the merge_manager path from the `local_dirs` in `ExecutorShuffleInfo`. It actually finds the parent dir of the block manager `local_dirs` and then assumes that `merge_manager` exists under it. The code is here:
   https://github.com/apache/spark/blob/12abfe79173f6ab00b3341f3b31cad5aa26aa6e4/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java#L1003
   This would require sending the merge_manager directory name to the server as well. This requires updating `RegisterExecutor` message but since we can't change an existing message, we will have to create a new one.
   
   I still think we can ignore this edge case because if the merged files are lost there will be a fallback to original blocks. There are 2 cases:
   1. Server hasn't finalized a shuffle yet and merged folder is deleted. When the shuffle is finalized, since the files are not there, the partition will not be considered merged and so original shuffle blocks will be fetched.
   2. Server has finalized and merged folder is deleted after that. In this case, during fetch of the merged partition by client, the server will respond with failure. This will cause the client to fallback on the original un-merged blocks.
   
   @zhouyejoe @mridulm Let me know what you think




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638325094



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)

Review comment:
       I think there is a race condition. pushMessage of an older attempt may still be able to create a new appShuffleInfo and partitions may still have an  old appAttemptShuffleId when there is a context switch.
   
   I think we need to also delete such appShuffleInfos during finalize. In finalization, we should remove all the appShuffleInfo that belong to older attempts of the app.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-856517863


   Added two unit tests for IndexShuffleBlockResolver getMergedBlockData and getMergedBlockMeta


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r620549792



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Yes, I think this would be a simpler option without changing the existing protocol.
   But I found it is not so straight forward to get the attemptID within the Executor. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648743212



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,7 +116,7 @@ public ShuffleIndexInformation load(File file) throws IOException {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleId appShuffleId,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId.appId, appShuffleId.shuffleId, reduceId);

Review comment:
       Will resolve the multi-attempts in SPARK-35546




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r616005406



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Also the issue @mridulm pointed out cannot be solved by either creating a random merge dir or creating it under a block manager directory. Spark shuffle server does **not** try to figure out which block mgr directories belong to a specific attempt and just delete those. In fact, it just leverages Yarn to delete the application local directories. There is a flag for cleaning up local directories in `blockHandler.applicationRemoved(...)`, however the flag is false when the `stopApplication` is invoked in `YarnShuffleService`.
   https://github.com/apache/spark/blob/d37d18dd7f628bfa84df2478c84ee52b089e7651/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java#L368
   
   So, this issue exists for the blockmgr dirs as well.  Either we create these dirs outside application local dirs which is managed by Yarn or I think this should be a fix in Yarn not in spark. Yarn should create app local dirs for an attempt and just delete those when an attempt fails.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-849395958


   @mridulm @Victsm @otterc @Ngone51  Updated the PR. Please help review. Thanks.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r641010452



##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       Wu Yi recommended to change to java.io._ in former comments so I changed it.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r604559381



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi

Review comment:
       Need to add `@Since("3.2.0")`
   

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi

Review comment:
       Need to add @Since("3.2.0")

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +92,37 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.appId, mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.appId, mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.appId, mergedMetaBlockId.name)
+      case _ =>
+        throw new RuntimeException(s"Only merged block ID is supported, but got ${blockId}")
+    }
+  }
+
+  private def getMergedShuffleFile(appId: String, filename: String): File = {

Review comment:
       This needs refactoring. On the server side, we have changed this to leverage `ExecutorDiskUtils.getFile`. I think we need to leverage that here as well.
   https://github.com/apache/spark/blob/46f96e9ce1cf998f521997bef1bd7367838f0d57/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java#L218
   

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"
+}
+
+@DeveloperApi
+case class ShuffleMergedMetaBlockId(

Review comment:
       Need to add `@Since("3.2.0")`

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"
+}
+
+@DeveloperApi
+case class ShuffleMergedMetaBlockId(
+  appId: String,

Review comment:
       Nit: indentation

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,

Review comment:
       Nit: indentation

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {

Review comment:
       If I remember correctly, the existing UT for this only tests for the parent merge_manager dir. Is it possible to extend that UT for checking the subdir creation?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)
+        val proc = builder.start()
+        val exitCode = proc.waitFor()
+        if (dirToCreate.exists()) {
+          created = dirToCreate
+        }
+        logDebug(
+          s"Created directory at ${dirToCreate.getAbsolutePath} and exitCode $exitCode")
+      } catch {
+        case e: SecurityException => created = null;
+      }
+    }
+  }
+
+

Review comment:
       Nit: extra line

##########
File path: core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
##########
@@ -85,6 +86,36 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
     assert(diskBlockManager.getAllBlocks().isEmpty)
   }
 
+  test("find active merged shuffle directories") {
+    testConf.set("spark.local.dir", rootDirs)
+    testConf.set("spark.shuffle.push.based.enabled", "true")
+    testConf.set("spark.shuffle.service.enabled", "true")
+    diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
+    assert(diskBlockManager.activeMergedShuffleDirs.isDefined)
+    assert(diskBlockManager.activeMergedShuffleDirs.get.length == diskBlockManager.localDirs.length)
+    val expected = Array(rootDir0.getAbsolutePath, rootDir1.getAbsolutePath).sorted
+    val actual = diskBlockManager.activeMergedShuffleDirs.get.map(file => file.getParent)
+    assert(expected sameElements actual)
+  }
+
+  test("should not create merge directories if one already exists under a local dir") {
+    val mergeDir0 = new File(rootDir0, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (!mergeDir0.exists()) {
+      Files.createDirectories(mergeDir0.toPath)
+    }
+    val mergeDir1 = new File(rootDir1, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (mergeDir1.exists()) {
+      Utils.deleteRecursively(mergeDir1)
+    }
+    testConf.set("spark.local.dir", rootDirs)
+    testConf.set("spark.shuffle.push.based.enabled", "true")
+    testConf.set("spark.shuffle.service.enabled", "true")
+    diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
+    assert(diskBlockManager.activeMergedShuffleDirs.isDefined)
+    assert(diskBlockManager.activeMergedShuffleDirs.get.length == 1)
+  }
+
+

Review comment:
       Nit: extra line

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -110,6 +110,22 @@ private[spark] class IndexShuffleBlockResolver(
       .getOrElse(blockManager.diskBlockManager.getFile(blockId))
   }
 
+  private def getMergedBlockDataFile(appId: String, shuffleId: Int, reduceId: Int): File = {
+    blockManager.diskBlockManager.getMergedShuffleFile(
+      ShuffleMergedBlockId(appId, shuffleId, reduceId))
+  }
+
+  private def getMergedBlockIndexFile(appId: String, shuffleId: Int, reduceId: Int): File = {
+    blockManager.diskBlockManager.getMergedShuffleFile(
+      ShuffleMergedIndexBlockId(appId, shuffleId, reduceId))
+  }
+
+  private def getMergedBlockMetaFile(appId: String, shuffleId: Int, reduceId: Int): File = {
+    blockManager.diskBlockManager.getMergedShuffleFile(
+      ShuffleMergedMetaBlockId(appId, shuffleId, reduceId))
+  }
+
+

Review comment:
       Nit: extra line

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return
+        }
+      }
+      // Since this executor didn't see any merge_manager directories, it will start creating them.
+      // It's possible that the other executors launched at the same time may also reach here but
+      // we are working on the assumption that the executors launched around the same time will
+      // have the same set of application local directories.
+      localDirs.flatMap { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // Only one container will create this directory. The filesystem will handle any race
+          // conditions.
+          if (!mergeDir.exists()) {
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))
+              Utils.createDirWith770(sudDir)
+            }
+          }
+          logInfo(s"Merge directory at $mergeDir")
+          Some(mergeDir)
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+            None
+        }
+      }
+      Utils.getConfiguredLocalDirs(conf).map(rootDir => new File(rootDir, MERGE_MANAGER_DIR))
+    }
+  }
+
+  private def findActiveMergedShuffleDirs(conf: SparkConf): Option[Array[File]] = {
+    Option(Utils.getConfiguredLocalDirs(conf).map(
+      rootDir => new File(rootDir, "merge_manager")).filter(mergeDir => mergeDir.exists()))
+  }
+
+

Review comment:
       Nit: extra line

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories

Review comment:
       Nit: change the comment also mention merge_manager + subdirs

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }
+    // Number of chunks is number of indexes - 1
+    val numChunks = size / 8 - 1
+    val chunkSizes = new Array[Long](numChunks)
+    for (index <- 0 until numChunks) {
+      chunkSizes(index) = offsets.get(index + 1) - offsets.get(index)
+    }
+    chunkSizes.indices.map {
+      index =>
+        new FileSegmentManagedBuffer(transportConf, dataFile,
+          offsets.get(index), chunkSizes(index))
+    }
+  }
+
+  /**
+   * This is only used for reading local merged block meta data.
+   */
+  override def getMergedBlockMeta(blockId: ShuffleBlockId): MergedBlockMeta = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val size = indexFile.length.toInt
+    val numChunks = (size / 8) - 1
+    val metaFile = getMergedBlockMetaFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val chunkBitMaps = new FileSegmentManagedBuffer(transportConf, metaFile, 0L, metaFile.length)
+    new MergedBlockMeta(numChunks, chunkBitMaps)
+  }
+
+

Review comment:
       Nit: extra line




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r639133435



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       Btw, we could eliminate need for `applications` and use `appsPathsInfo` instead (and use partitions.compute in the if block above).




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644436170



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }

Review comment:
       The original logic:
   For loop each local dir, if there is any merge_dir created by other executors, this executor will not create any local dirs.
   Potential issue:
   Suppose Executor 1 created the merge_dirs under /tmp/[a-c]. Executor2 launched slightly later, it got local dirs /tmp/[b-d]. Executor2 would not create merge_dirs in any of them as it found that another executor has created the merge dir in /tmp/b. But if the executor registration message from Executor2 gets handled prior to Executor1's, shuffle service will use the /tmp/[b-d] as the merge dirs. However, the merge_dir under /tmp/d has not been created by Executor2.
   
   The update logic:
   Every Executor should try to create the merge_dir under the local dirs, which makes sure no matter what ExecutorRegister message gets received in shuffle service, it is guaranteed that the merge_dir is there with permission 770.
   @otterc
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-846476480


   @otterc @mridulm @Ngone51 Addressed the comments. Please review. Thanks.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627553728



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       You can always have a race between creation and deletion, right ?
   a) Executor e1.attempt2 checks - found no directories.
   b) Executor e2.attempt1 checks - found no directories
   c) Both create directories.
   d) Both register to ESS - now ESS sees two directories.
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r647789581



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This will be handled as part of SPARK-35546, resolving thread.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648817463



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"$MERGE_MANAGER_DIR dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {

Review comment:
       I'm ok with it. And I think we need a more specific name after taking away the `permission` parameter.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644519782



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       Similarly to original shuffle fetch, in ShuffleBlockResolver and IndexShuffleBlockResolver, "Local" is not added to the method name. But in BlockManager, we have getLocalBlockData and getHostLocalShuffleData.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859045755


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/139658/
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637975592



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -190,3 +295,9 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 }
+
+private[spark] object DiskBlockManager {
+  private[spark] val MERGE_DIRECTORY = "merge_directory"
+  private[spark] val MERGE_DIR_KEY = "mergeDir"
+  private[spark] val ATTEMPT_ID_KEY = "attemptId"

Review comment:
       The object is already private spark. These modifiers look redundant.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -190,3 +295,9 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 }
+
+private[spark] object DiskBlockManager {
+  private[spark] val MERGE_DIRECTORY = "merge_directory"
+  private[spark] val MERGE_DIR_KEY = "mergeDir"
+  private[spark] val ATTEMPT_ID_KEY = "attemptId"

Review comment:
       The object is already private spark. These modifiers look redundant.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627646535



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To add to my comment above (should have provided more context): I was assuming we are relying on directory existence to infer latest.
   But I like @Ngone51's idea here better:
   >  And if the executor becomes the one who creates the merge dir, we send the ExecutorShuffleInfo with the special shuffleManager, e.g., "sort_merge_manager_attemptX". And ExternalBlockHandler can parse the shuffleManager into two parts
   
   `shuffleManager` should to be relevant when we had multiple shuffle managers - and from evolution point of view, we could always introduce a new shuffle in future.
   My proposal is extension of the idea above - here, attempt is metadata about shuffleManager we want to convey.
   Ideally, this should be within `ExecutorShuffleInfo` - but given the compatibility issues here, why not add a general way to encode metadata about the shuffle manager here ?
   
   That is, instead of hardcoding 'SortShuffleManager_attemptX' and 'SortShuffleManager' as supported patterns, we could simply allow for 'SortShuffleManager' (existing) and 'SortShuffleManager:json_string' ?
   This can allow for any potential future evolution to also be possible - with currently merge_dir: "merge_directory_<attemptid>" being the initial field (@zhouyejoe let us explicitly pass the directory name, instead of inferring it from attempt id ?)
   
   Ofcourse if ':' is missing in shuffleManager, then we treat it as empty metadata case.
   
   With this in place, we would still need changes to push block protocol to include attempt id - but given no one is using that currently, we can make that change.
   With this explicit specification of attempt id by executor, we remove all ambiguity.
   
   Thoughts @Ngone51 ?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640879804



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -336,51 +436,26 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
       && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
         : partitionInfoBeforeCheck;
-    final String streamId = String.format("%s_%d_%d_%d",
-      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
-      msg.reduceId);
     if (partitionInfo != null) {
       return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
     } else {
       // For a duplicate block or a block which is late, respond back with a callback that handles
       // them differently.
-      return new StreamCallbackWithID() {
-        @Override
-        public String getID() {
-          return streamId;
-        }
-
-        @Override
-        public void onData(String streamId, ByteBuffer buf) {
-          // Ignore the requests. It reaches here either when a request is received after the
-          // shuffle file is finalized or when a request is for a duplicate block.
-        }
-
-        @Override
-        public void onComplete(String streamId) {
-          if (isTooLate) {
-            // Throw an exception here so the block data is drained from channel and server
-            // responds RpcFailure to the client.
-            throw new RuntimeException(String.format("Block %s %s", streamId,
-              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
-          }
-          // For duplicate block that is received before the shuffle merge finalizes, the
-          // server should respond success to the client.
-        }
-
-        @Override
-        public void onFailure(String streamId, Throwable cause) {
-        }
-      };
+      if (isTooLate) {

Review comment:
       Can you please explain why you have broken down this into if/else here? What will will the client do when it receives a `null` as a failure message? This again deviates from the goal of this PR.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-850024225


   In fact, I'm leaning towards taking both handling multiple app attempts and the race conditions out of this PR and fix them in a separate jira.
   The original scope of this ticket is to add support in DiskBlockManager to handle merge directories.
   We can still keep the app attempt ID in the merge dir directory path in this PR.
   However, for handling the registration of executors with a new attempt ID and the cleanup of metadata associated with the previous attempt ID, I think separating these changes out in a dedicated jira makes sense.
   With SPARK-32922 that @otterc is working on, we will be able to conduct end-to-end test/benchmark for the version of push-based shuffle that's currently being contributed upstream.
   I prefer to test these changes associated with handling multiple app attempts and the race conditions in such end-to-end test/benchmarks before committing the changes.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-823463844


   @tgravescs @Ngone51 @attilapiros Can you help review this PR? Thanks.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638440984



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       Thinking more, while the above minimizes the possibility of leak, it does not theoretically eliminate it.
   
   For example:
   
   If no entry in partitions map for shuffle:
   a) T1: getOrCreateAppShufflePartitionInfo -> partitions.computeIfAbsent -> mappingFunction lambda
   c) T1: `synchronized(applications) { if (applications.contains(appId))  { return id -> Maps.newConcurrentMap() } else reutrn null; }` -> returns new map.
   d) Context switch to T2.
   e) T2: applicationRemoved -> applications.remove(appId) -> finish iterating over partitions map.
   f) T1: partitions completes adding to map.
   
   Now we have an entry in partitions map which will never be cleaned up.
   A solution to this would be something like:
   
   ```
   Map<Integer, AppAttemptShufflePartitionInfo> shufflePartitions = partitions.get(appAttemptShuffleId);
   if (null == shufflePartitions) {
     synchronized(applications) {
       if (applications.contains(appId)) {
         shufflePartitions = partitions.computeIfAbsent(appAttemptShuffleId, id -> Maps.newConcurrentMap());
       }
     }
   }
   ```
   
   
   This minimizes cost of synchronized to only when shuffle is not known for app attempt - while ensuring correctness.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r616005406



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Also the issue @mridulm pointed out cannot be solved by either creating a random merge dir or creating it under a block manager directory. Spark shuffle server does **not** try to figure out which block mgr directories belong to a specific attempt and just delete those. In fact, it just leverages Yarn to delete the application local directories. There is a flag for cleaning up local directories in `blockHandler.applicationRemoved(...)`, however the flag is false when the `stopApplication` is invoked in `YarnShuffleService`.
   https://github.com/apache/spark/blob/d37d18dd7f628bfa84df2478c84ee52b089e7651/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java#L368
   
   So, this issue exists for the blockmgr dirs as well.  Either we create these dirs outside application local dirs which are managed by Yarn or I think this should be a fix in Yarn not in spark. Yarn should create app local dirs for an attempt and just delete those when an attempt fails.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858860611


   Can you update to the latest master @zhouyejoe 


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r621391061



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This is a good point, unfortunately we do not set the attempt id in spark conf like we do for app id.
   A few options here would be:
   
   a) Also propagate attempt id via "spark.app.attemptId" is available (and use default value if missing).
   b) Defer registeration/directory creation to first task being run : ShuffleMapTask has `appAttemptId` as part of it. 
   c) For yarn, CONTAINER_ID env variable can be parsed to fetch attempt id - though this might not be optimal.
   
   I am not very keen on modifying protocol if possible.
   
   Thoughts ?
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r604563989



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories

Review comment:
       Nit: change the comment also to mention merge_manager + subdirs




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r619966403



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To minimize change, why not get the executor to create the directory as "merge_directory_<attempt_id>" ? (if missing)
   While registering, ESS can pick up the highest attempt id and associate the executor with that attempt id (and so merge_directory).
   
   Thoughts ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638422896



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt.
+          // Former attempt's shuffle partitions information will also be cleaned up.
+          boolean newAttemptRegistered = false;
+          if (appsPathsInfo.containsKey(appId)
+              && mergeDirectoryMeta.attemptId > appsPathsInfo.get(appId).attemptId) {
+            newAttemptRegistered = true;
+          }
+          appsPathsInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo == null
+                || (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId)) {
+              appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
+                executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+            }
+            return appAttemptPathsInfo;
+          });
+          // It is safe to clean up the AppShufflePartitionInfo
+          if (newAttemptRegistered) {
+            cleanupShufflePartitionInfo(appId, mergeDirectoryMeta.attemptId);
+          }
+        }
+      } catch (JsonProcessingException e ) {
+        logger.warn("Failed to get the merge directory information from ExecutorShuffleInfo: ", e);
+      }
+    } else {
+      logger.warn("ExecutorShuffleInfo does not have the expected merge directory information");
     }
-    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
-      executorInfo.subDirsPerLocalDir));
   }
-  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
-    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
-      reduceId);
+
+  private static String generateFileName(String appId, int shuffleId, int reduceId) {
+    return String.format("shuffleMerged__%s_%d_%d", appId, shuffleId, reduceId);

Review comment:
       Should be "shuffleMerged_%s_%d_%d". 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648715501



##########
File path: core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala
##########
@@ -161,4 +166,78 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa
     val resolver = new IndexShuffleBlockResolver(conf, blockManager)
     assert(resolver.getMigrationBlocks(ShuffleBlockInfo(Int.MaxValue, Long.MaxValue)).isEmpty)
   }
+
+  test("getMergedBlockData should return expected FileSegmentManagedBuffer list") {
+    val shuffleId = 1
+    val reduceId = 1
+    val dataFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.data"
+    val dataFile = new File(tempDir.getAbsolutePath, dataFileName)
+    val out = new FileOutputStream(dataFile)
+    Utils.tryWithSafeFinally {
+      out.write(new Array[Byte](30))
+    } {
+      out.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val managedBufferList =
+      resolver.getMergedBlockData(ShuffleBlockId(shuffleId, -1, reduceId), dirs)
+    assert(managedBufferList.size === 3)
+    assert(managedBufferList(0).size === 10)
+    assert(managedBufferList(1).size === 0)
+    assert(managedBufferList(2).size === 20)
+  }
+
+  test("getMergedBlockMeta should return expected MergedBlockMeta") {
+    val shuffleId = 1
+    val reduceId = 1
+    val metaFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.meta"
+    val metaFile = new File(tempDir.getAbsolutePath, metaFileName)
+    val chunkTracker = new RoaringBitmap()
+    chunkTracker.add(1)
+    chunkTracker.add(2)
+    val metaFileOutputStream = new FileOutputStream(metaFile)
+    val outMeta = new DataOutputStream(metaFileOutputStream)
+    Utils.tryWithSafeFinally {
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(3)
+      chunkTracker.add(4)
+      chunkTracker.serialize(outMeta)
+      chunkTracker.clear()
+      chunkTracker.add(5)
+      chunkTracker.add(6)
+      chunkTracker.serialize(outMeta)
+    }{
+      outMeta.close()
+    }
+    val indexFileName = s"shuffleMerged_${appId}_${shuffleId}_$reduceId.index"
+    prepareMergedShuffleIndexFile(indexFileName)
+    val resolver = new IndexShuffleBlockResolver(conf, blockManager)
+    val dirs = Some(Array[String](tempDir.getAbsolutePath))
+    val mergedBlockMeta =
+      resolver.getMergedBlockMeta(ShuffleBlockId(shuffleId, -1, reduceId), dirs)
+    assert(mergedBlockMeta.getNumChunks === 3)
+    assert(mergedBlockMeta.readChunkBitmaps().size === 3)
+    assert(mergedBlockMeta.readChunkBitmaps()(0).contains(1))
+    assert(mergedBlockMeta.readChunkBitmaps()(0).contains(2))

Review comment:
       Added checks, and also check for invalid mapIds in the bitmap.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645338885



##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       You did recommend this change in former reviews. https://github.com/apache/spark/pull/32007#discussion_r622840432




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615488297



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }
+      if (!mergeDirCreated) {
+        // This executor didn't see any merge_manager directories, it will start creating them.
+        // It's possible that the other executors launched at the same time may also reach here but
+        // we are working on the assumption that the executors launched around the same time will
+        // have the same set of application local directories.
+        localDirs.foreach { rootDir =>
+          try {
+            val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+            // Only one container will create this directory. The filesystem will handle any race
+            // conditions.
+            if (!mergeDir.exists()) {
+              Utils.createDirWith770(mergeDir)

Review comment:
       @otterc I added the merge dir creation first here, so I didn't add the "-p" in the createDirWith770 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640999074



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,64 +123,85 @@ public ShuffleIndexInformation load(File file) throws IOException {
    * shuffle does not exist, initializes the metadata.
    */
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
-      AppShuffleId appShuffleId,
+      AppAttemptShuffleId appAttemptShuffleId,
+      AppAttemptPathsInfo appAttemptPathsInfo,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
-    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+    File dataFile = getMergedShuffleDataFile(
+      appAttemptPathsInfo, appAttemptShuffleId.appId, appAttemptShuffleId.shuffleId, reduceId);
+    if (!partitions.containsKey(appAttemptShuffleId) && dataFile.exists()) {
       // If this partition is already finalized then the partitions map will not contain
       // the appShuffleId but the data file would exist. In that case the block is considered late.
       return null;
     }
-    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
-      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
-    return shufflePartitions.computeIfAbsent(reduceId, key -> {
-      // It only gets here when the key is not present in the map. This could either
-      // be the first time the merge manager receives a pushed block for a given application
-      // shuffle partition, or after the merged shuffle file is finalized. We handle these
-      // two cases accordingly by checking if the file already exists.
-      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
-      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
-      try {
-        if (dataFile.exists()) {
-          return null;
-        } else {
-          return newAppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
-        }
-      } catch (IOException e) {
-        logger.error(
-          "Cannot create merged shuffle partition with data file {}, index file {}, and "
-            + "meta file {}", dataFile.getAbsolutePath(),
-            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
-        throw new RuntimeException(
-          String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s "
-          + "reduceId %s", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+    // While processing the application remove, where the shuffle partitions info for the specific
+    // application will be cleaned up, this method will still be called to create new partitions
+    // as of receiving the push blocks. To avoid the potential memory leak, before creating the
+    // empty hashmap for storing the shuffle partitions information in the partitions hashmap,
+    // we need to make sure that the entry for the specific application must still exist in
+    // appAttemptsPathInfo hashmap. Otherwise, the push blocks should be ignored.

Review comment:
       Discussed offline. Even though when the context switch happens after line 150, during applicationRemove, the entry for this key/value pair will be removed from the partitions hashmap. The shufflePartitions AtomicReference won't be null and files will be created for merge shuffle. But the newly created hashmap will be a temporary referenced object in this method only and it will be cleaned by JVM itself.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642024816



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +349,64 @@ void deleteExecutorDirs(Path[] dirs) {
     }
   }
 
+  /**
+   * Create StreamCallback for invalid push blocks with the specific error message.
+   * If specific error message is null, this StreamCallback won't throw exception in client.
+   */
+  private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+      String streamId,
+      String errorMessage) {
+    return new StreamCallbackWithID() {
+      @Override
+      public String getID() {
+        return streamId;
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) {
+        // Ignore the requests. It reaches here either when a request is received after the
+        // shuffle file is finalized or when a request is for a duplicate block.
+      }
+
+      @Override
+      public void onComplete(String streamId) {
+        if (errorMessage != null) {
+          // Throw an exception here so the block data is drained from channel and server
+          // responds RpcFailure to the client.
+          throw new RuntimeException(String.format("Block %s %s", streamId, errorMessage));
+        }
+        // For duplicate block that is received before the shuffle merge finalizes, the
+        // server should respond success to the client.
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable cause) {
+      }
+    };
+  }
+
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppAttemptPathsInfo appAttemptPathsInfo = getAppAttemptPathsInfo(msg.appId);
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId =
+      new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+      // If this Block belongs to a former application attempt, it is considered late,
+      // as only the blocks from the current application attempt will be merged
+      // TODO: [SPARK-35548] Client should be updated to handle this error.
+      return createCallbackForInvalidPushBlocks(streamId,

Review comment:
       Is there a need to create a valid `StreamCallback` in this case? Can we just not throw the RuntimeException with the `NEWER_ATTEMPT_HAS_STARTED_MESSAGE_SUFFIX` when it tries to create the stream? This will also make the refactoring  below which added `createCallbackForInvalidPushBlocks` unnecessary. 
   We do throw RuntimeException when `Cannot initialize merged shuffle partition for appId %s shuffleId %s"reduceId %s` from `getOrCreateAppShufflePartitionInfo`, so we can immediately fail to create the stream here.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858866677


   jenkins, test this please


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644969959



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (dirs.map(_.nonEmpty).getOrElse(false)) {
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638021677



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -529,10 +530,17 @@ private[spark] class BlockManager(
 
   private def registerWithExternalShuffleServer(): Unit = {
     logInfo("Registering executor with local external shuffle service.")
+    val shuffleManagerMeta =
+      if (conf.get(config.PUSH_BASED_SHUFFLE_ENABLED)) {
+        s"${shuffleManager.getClass.getName}:" +
+          s"${diskBlockManager.getMergeDirectoryAndAttemptIDJsonString()}}}"

Review comment:
       We shall add comments to `ExecutorShuffleInfo` to explain the `shuffleManagerMeta`.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648739027



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"$MERGE_MANAGER_DIR dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {

Review comment:
       Nit: Since this is moved to DiskBlockManager now and is no more a utility, will it make sense to not pass in permission and just make this specific for creating a dir with permission `770`?  I say this again because for restrictive permissions, we shouldn't be executing the command `mkdir`. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638036135



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt.
+          // Former attempt's shuffle partitions information will also be cleaned up.
+          boolean newAttemptRegistered = false;
+          if (appsPathsInfo.containsKey(appId)
+              && mergeDirectoryMeta.attemptId > appsPathsInfo.get(appId).attemptId) {
+            newAttemptRegistered = true;
+          }
+          appsPathsInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo == null
+                || (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId)) {
+              appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
+                executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+            }
+            return appAttemptPathsInfo;
+          });
+          // It is safe to clean up the AppShufflePartitionInfo
+          if (newAttemptRegistered) {
+            cleanupShufflePartitionInfo(appId, mergeDirectoryMeta.attemptId);

Review comment:
       nit: add a warning log for this case?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637984727



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -419,4 +419,8 @@ public long mergedIndexCacheSize() {
   public int ioExceptionsThresholdDuringMerge() {
     return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
   }
+
+  public int appAttemptId() {
+    return conf.getInt("spark.app.attempt.id", -1);

Review comment:
       I think this's now is only valid for YARN. Could you comment it?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642149224



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       Should we add "local" in other places too?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614589850



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This should be sufficient as there would be only one attempt running at the same time. If one attempt fails to run, the local directories will be removed. So even though different attempts are using the same tmp path, it won't be problematic.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627646535



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To add to my comment above (should have provided more context): I was assuming we are relying on directory existence to infer latest.
   But I like @Ngone51's idea here better:
   >  And if the executor becomes the one who creates the merge dir, we send the ExecutorShuffleInfo with the special shuffleManager, e.g., "sort_merge_manager_attemptX". And ExternalBlockHandler can parse the shuffleManager into two parts
   
   `shuffleManager` should to be relevant when we had multiple shuffle managers - and from evolution point of view, we could always introduce a new shuffle in future.
   My proposal is extension of the idea above - here, attempt is metadata about shuffleManager we want to convey.
   Ideally, this should be within `ExecutorShuffleInfo` - but given the compatibility issues here, why not add a general way to encode metadata about the shuffle manager here ?
   
   That is, instead of hardcoding 'SortShuffleManager_attemptX' and 'SortShuffleManager' as supported patterns, we could simply allow for 'SortShuffleManager' (existing) and 'SortShuffleManager:json_string' ?
   This can allow for any potential future evolution to also be possible - with currently `{"merge_dir": "merge_directory_<attemptid>", "attempt_id":<attempt_id>}` being the initial field (@zhouyejoe let us explicitly pass the directory name, instead of inferring it from attempt id ?)
   
   Ofcourse if ':' is missing in shuffleManager, then we treat it as empty metadata case.
   
   With this in place, we would still need changes to push block protocol to include attempt id - but given no one is using that currently, we can make that change.
   With this explicit specification of merge directory, we remove all ambiguity.
   
   Thoughts @Ngone51, @otterc, @zhouyejoe  ?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614514919



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       The merged shuffle files are already written by the server and the file identifier doesn't include the attemptId. To include attemptId we do require a change on the push side as well where attempt Id is also included in the PushBlockMessage and the server uses that to create merged files. I think this is a much bigger change and we have this jira [SPARK-32923 ](https://issues.apache.org/jira/browse/SPARK-32923) to address it.
   
   One thing we can do here is that we introduce attemptId in these new Ids that we are creating but not use them while trying to find file and add a TODO.
   What do you think @mridulm @Victsm @zhouyejoe ?
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640349087



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       @mridulm FYI, I have not added the unit test for this memory leak yet in this update PR. Post the majority of the required change first and I will add it tomorrow.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r624432860



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Btw, forgot to add my thoughts on a new rpc message.
   I am referring to adding something like `RegisterExecutorForPushBasedShuffle` (or some such) in addition to existing `RegisterExecutor`. That is, send `RegisterExecutorForPushBasedShuffle` after `RegisterExecutor` succeeds.
    
   A few thoughts:
   
   * If ESS does not support the new RPC, how is the spark application supposed to behave ?
     * Case where `RegisterExecutor` would succeed while `RegisterExecutorForPushBasedShuffle` would fail due to `IllegalArgumentException` at ESS due to unrecognized msg id.
     * Currently, it would throw a `SparkException` and leads to executor failure : do we change this behavior ? Or do we simply fail the application due to unsupported config ?
   * Adding the new rpc allows us to decouple executor registration from whether the executor host should be candidate for hosting mergers or not.
     * This will help with future evolution.
   * If we are taking this path, it would be better for ESS to manage the merger location entirely - and not have executors create/update it (as discussed above). It will help ESS evolve independently.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-813837901


   +CC @tgravescs, @Ngone51, @attilapiros


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859033409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/44186/
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645069135



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -129,6 +155,9 @@ object BlockId {
   val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r
   val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r
   val SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r
+  val SHUFFLE_MERGED = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).data".r
+  val SHUFFLE_MERGED_INDEX = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).index".r
+  val SHUFFLE_MERGED_META = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).meta".r

Review comment:
       Just figured it out, this is for unit testing, similarly to other unit tests for case classes defined in BlockId.scala




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645805428



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&

Review comment:
       Not in this core module, and I am seeing "yarn" used in lots other places.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r624235902



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       cc @Victsm 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r647807818



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -149,6 +178,12 @@ object BlockId {
       ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt)
     case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) =>
       ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED(appId, shuffleId, reduceId) =>

Review comment:
       Oh, yes, I was trying to go over each former comments and see whether it has been resolved, as the PR got reverted to some former commit. Nice catch. Thanks.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644689361



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {

Review comment:
       nit: "createMergeManagerDirIfNeeded()" ?
   
   btw: `conf` can be omitted as it's a member field.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to

Review comment:
       nit: `shuffle service` -> `external shuffle service`

##########
File path: core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
##########
@@ -85,6 +86,24 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
     assert(diskBlockManager.getAllBlocks().isEmpty)
   }
 
+  test("should still create merge directories if one already exists under a local dir") {
+    val mergeDir0 = new File(rootDir0, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (!mergeDir0.exists()) {
+      Files.createDirectories(mergeDir0.toPath)
+    }
+    val mergeDir1 = new File(rootDir1, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (mergeDir1.exists()) {
+      Utils.deleteRecursively(mergeDir1)
+    }
+    testConf.set("spark.local.dir", rootDirs)
+    testConf.set("spark.shuffle.push.enabled", "true")
+    testConf.set("spark.shuffle.service.enabled", "true")
+    diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
+    Utils.getConfiguredLocalDirs(testConf).map(
+      rootDir => new File(rootDir, DiskBlockManager.MERGE_MANAGER_DIR))
+      .filter(mergeDir => mergeDir.exists())

Review comment:
       Add assertions?

##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleReadingSuite.scala
##########
@@ -133,4 +133,13 @@ class HostLocalShuffleReadingSuite extends SparkFunSuite with Matchers with Loca
       assert(remoteBytesRead.sum === 0 && remoteBlocksFetched.sum === 0)
     }
   }
+
+  test("Enable host local shuffle reading when Push based shuffle is enabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_SERVICE_ENABLED, true)
+      .set("spark.yarn.maxAttempts", "1")
+      .set(PUSH_BASED_SHUFFLE_ENABLED, true)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test-host-local-shuffle-reading", conf)

Review comment:
       nit: `local-cluster[2, 1, 1024]` (extra space after the comma)

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       IIUC, I think `dirs.get.isEmpty` is what you really need here.

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +370,51 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  override def getMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Option[Array[String]]): Seq[ManagedBuffer] = {
+    val indexFile =
+      getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }
+    // Number of chunks is number of indexes - 1
+    val numChunks = size / 8 - 1
+    for (index <- 0 until numChunks) yield {
+      new FileSegmentManagedBuffer(transportConf, dataFile,
+        offsets.get(index),
+        offsets.get(index + 1) - offsets.get(index))
+    }
+  }
+
+  /**
+   * This is only used for reading local merged block meta data.
+   */
+  override def getMergedBlockMeta(
+      blockId: ShuffleBlockId,
+      dirs: Option[Array[String]]): MergedBlockMeta = {
+    val indexFile =
+      getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val size = indexFile.length.toInt
+    val numChunks = (size / 8) - 1
+    val metaFile = getMergedBlockMetaFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val chunkBitMaps = new FileSegmentManagedBuffer(transportConf, metaFile, 0L, metaFile.length)
+    new MergedBlockMeta(numChunks, chunkBitMaps)
+  }
+
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */

Review comment:
       Does this comment belong to `getMergedBlockData`?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       Why not use `dirs: Option[Array[String]]` directly?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-p", "-m" + permission, dirToCreate.getAbsolutePath)
+        val proc = builder.start()
+        val exitCode = proc.waitFor()
+        if (dirToCreate.exists()) {
+          created = dirToCreate
+        }
+        logDebug(
+          s"Created directory at ${dirToCreate.getAbsolutePath} and exitCode $exitCode")
+      } catch {
+        case e: SecurityException => created = null;

Review comment:
       Add a warning log with the exception?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       I think this comment is not appropriate for a util function. Shall we move this to the caller side in `DiskBlockManager`?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +729,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def getLocalMergedBlockMeta(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): MergedBlockMeta = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId, Some(dirs))

Review comment:
       How do you plan test `getLocalMergedBlockData` and `getLocalMergedBlockMeta`?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // This executor does not find merge_manager directory, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.
+          if (!mergeDir.exists()) {
+            logDebug(
+              s"Try to create $mergeDir and its sub dirs since the merge dir does not exist")

Review comment:
       nit: `merge` -> `merge_manager`?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
##########
@@ -35,7 +35,11 @@
   public final String[] localDirs;
   /** Number of subdirectories created within each localDir. */
   public final int subDirsPerLocalDir;
-  /** Shuffle manager (SortShuffleManager) that the executor is using. */
+  /** Shuffle manager (SortShuffleManager) that the executor is using.

Review comment:
       ```suggestion
     /** 
      * Shuffle manager (SortShuffleManager) that the executor is using.
   ```

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +

Review comment:
       nit: `Failed to create directory ${dirToCreate.getAbsolutePath} with $permission...`?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)

Review comment:
       nit: 2 indents

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -431,7 +432,7 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
       executorInfo.subDirsPerLocalDir));
   }
   private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
-    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+    return String.format("shuffleMerged_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,

Review comment:
       Shall we make this prefix a constant field?

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -1438,16 +1441,34 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
     assert(message.contains(expected))
   }
 
-  test("isPushBasedShuffleEnabled when both PUSH_BASED_SHUFFLE_ENABLED" +
-    " and SHUFFLE_SERVICE_ENABLED are true") {
+  test("isPushBasedShuffleEnabled when PUSH_BASED_SHUFFLE_ENABLED " +
+    "and SHUFFLE_SERVICE_ENABLED are both set to true in YARN mode with maxAttempts set to 1") {
     val conf = new SparkConf()
     assert(Utils.isPushBasedShuffleEnabled(conf) === false)
     conf.set(PUSH_BASED_SHUFFLE_ENABLED, true)
     conf.set(IS_TESTING, false)
     assert(Utils.isPushBasedShuffleEnabled(conf) === false)
     conf.set(SHUFFLE_SERVICE_ENABLED, true)
+    conf.set(SparkLauncher.SPARK_MASTER, "yarn")
+    conf.set("spark.yarn.maxAttempts", "1")
     assert(Utils.isPushBasedShuffleEnabled(conf) === true)
+    conf.set("spark.yarn.maxAttempts", "2")
+    assert(Utils.isPushBasedShuffleEnabled(conf) === false)
+  }
+
+  test("Test create dir with 770") {
+    val testDir = new File("target/testDir");
+    FileUtils.deleteQuietly(testDir)
+    Utils.createDirWithCustomizedPermission(testDir, "770")
+    val permission = PosixFilePermissions.toString(
+      JavaFiles.getPosixFilePermissions(Paths.get("target/testDir")))
+    assert(permission.equals("rwxrwx---"))
+    val foo = new File(testDir, "foo.txt")
+    Files.touch(foo)
+    assert(testDir.exists && testDir.isDirectory)
+    FileUtils.deleteQuietly(testDir)
   }
+

Review comment:
       Revert this?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.

Review comment:
       Have you addressed this comment?
   
   Looking at the code, it still looks inconsistent with the comment. I assume the comment should look like "Will create the merge_manager directory if it doesn't exist under the local dir." to match the code.
   
   

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       I don't remember I recommended ever...but yes it's recommended to use wildcard imports when [there're more than  6 entities](https://github.com/databricks/scala-style-guide#imports).
   
   

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       This is not resolved? @zhouyejoe 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640330633



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       Added as suggested.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642149120



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +198,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {

Review comment:
       This change is for the code refactor as Mridul commented earlier.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615457180



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,37 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(

Review comment:
       Discussed with @zhouyejoe offline. The command is missing a `-p` to create the parent directory if it doesn't exist.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614521383



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       @mridulm I've tried this last year and it doesn't work. It still creates the directory with permission `750`.  Internally we have this  TODO as well for this method.
   ```
      * TODO: Find out why can't we create a dir using java api with permission 770
      *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
      *  PosixFilePermissions.fromString("rwxrwx---")))
      ```
   This has something to do with yarn setting a umask or setgid to something when it starts the container process. I don't remember the details because I tried this early last year.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637981047



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -40,6 +46,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   /* Create one local directory for each path mentioned in spark.local.dir; then, inside this
    * directory, create multiple subdirectories that we will hash files into, in order to avoid
    * having really large inodes at the top level. */
+  private val configuredLocalDirs: Array[String] = Utils.getConfiguredLocalDirs(conf)

Review comment:
       Does the comment above is applied to this new line too? If not, could you move this new line on top of the comment?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638009350



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -529,10 +530,17 @@ private[spark] class BlockManager(
 
   private def registerWithExternalShuffleServer(): Unit = {
     logInfo("Registering executor with local external shuffle service.")
+    val shuffleManagerMeta =
+      if (conf.get(config.PUSH_BASED_SHUFFLE_ENABLED)) {

Review comment:
       nit: use `Utils.isPushBasedShuffleEnabled(conf)` too?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648874572



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+      val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+      sparkMaxAttempts match {
+        case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+        case None => yarnMaxAttempts
+      }

Review comment:
       Added comment that this method will be removed after SPARK-35546




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858861907


   Updated to the latest master


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859108418


   Thanks for working on this @zhouyejoe
   Thanks for all the reviews @Ngone51, @Victsm, @otterc !


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644977102



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+      val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+      sparkMaxAttempts match {
+        case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+        case None => yarnMaxAttempts
+      }

Review comment:
       This already exists in spark right ? If yes, remove duplication ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-855593936


   Address most of the review comments. The unit test for getLocalMergedBlockData and getLocalMergedBlockMeta are yet to be added. I think we should add the unit tests for IndexShuffleBlockResolver for these two tests. 


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r645340869



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+      val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+      sparkMaxAttempts match {
+        case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+        case None => yarnMaxAttempts
+      }

Review comment:
       The code does exist in Spark code base, but in the module of resource-manager/yarn. Should we move this part to Utils.scala and then call it in YarnRMClient? Or we can just make a code duplicate here?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-853615497


   Updated with a slim version, which excludes the handling for multiple attempts case. 
   @Ngone51  Would like to share a little bit more context. We had multiple round of discussion internally regarding this PR, and the agreement we have reached internally is to exclude the multiple attempts support from this PR, whereas we have created a ticket SPARK-30602 to add it later on.
   
   cc @mridulm @Victsm @otterc


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638402695



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {

Review comment:
       Pass the attempt id for the request along here - and validate that `AppAttemptPathsInfo` fetched from `appsPathsInfo` is for the right attempt.
   Else we can end up in race between initial validation and `getFile`.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt.
+          // Former attempt's shuffle partitions information will also be cleaned up.
+          boolean newAttemptRegistered = false;
+          if (appsPathsInfo.containsKey(appId)
+              && mergeDirectoryMeta.attemptId > appsPathsInfo.get(appId).attemptId) {
+            newAttemptRegistered = true;
+          }
+          appsPathsInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo == null
+                || (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId)) {
+              appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
+                executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+            }
+            return appAttemptPathsInfo;
+          });
+          // It is safe to clean up the AppShufflePartitionInfo
+          if (newAttemptRegistered) {

Review comment:
       We should not rely on `newAttemptRegistered` here - but check if `compute` actually added an entry.
   For example, something like this:
   
   ```suggestion
             AtomicBoolean newAttemptRegistered = new AtomicBoolean(false);
             appsPathsInfo.compute(appId, (id, appAttemptPathsInfo) -> {
               if (appAttemptPathsInfo == null
                   || (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId)) {
                 appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
                   executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
                 newAttemptRegistered.set(true);
               }
               return appAttemptPathsInfo;
             });
             // It is safe to clean up the AppShufflePartitionInfo
             if (newAttemptRegistered.get()) {
   ```

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {

Review comment:
       Add a constant for `-1` (and probably change from `-1` given we have unknown attempt id as `-1` already ?) here to describe what behavior is expected (remove all attempts for application).

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -644,15 +644,15 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
 
     val tempDir2 = Utils.createTempDir()
     val sourceFile1 = new File(tempDir2, "foo.txt")
-    Files.touch(sourceFile1)
+    com.google.common.io.Files.touch(sourceFile1)

Review comment:
       Can we revert these changes ? Does not look related to this PR

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {

Review comment:
       Also, it should be `appAttemptShuffleId.attemptId < attemptId` ?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java
##########
@@ -45,18 +45,21 @@
 
   private final TransportClient client;
   private final String appId;
+  private final int attemptId;
   private final String[] blockIds;
   private final BlockFetchingListener listener;
   private final Map<String, ManagedBuffer> buffers;

Review comment:
       Review note: while a backwardly incompatible change between 3.1 and 3.2 - current codepaths in ESS 3.1 cant/dont rely on it, so this change should be fine.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       There is a race between this cleanup and some executor adding an entry into partitions map.
   In a nutshell, we need to maintain an `applications` `Set` which contains currently running apps.
   a) In `registerExecutor`, add to this `Set`
   b) In `applicationRemoved`, remove from this `Set` before doing `cleanupShufflePartitionInfo`.
   c) In `getOrCreateAppShufflePartitionInfo`, in `partitions.computeIfAbsent`, check if application is in this `Set` before returning a valid `Map` - else return `null` (and handle a `null` value for `shufflePartitions`).
   
   This handles the race condition of `getOrCreateAppShufflePartitionInfo` (from a remote executor) overlapping with `applicationRemoved` (on local NM).
   This also means the race condition @otterc referenced [above](https://github.com/apache/spark/pull/32007/files#r638325094) will not occur (application removal will ensure all records are cleaned up - so stale entries might live utmost until application termination - not after).
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r622794341



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;

Review comment:
       nit:  unnecessary ";" 

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }
+      if (!mergeDirCreated) {
+        // This executor didn't see any merge_manager directories, it will start creating them.
+        // It's possible that the other executors launched at the same time may also reach here but
+        // we are working on the assumption that the executors launched around the same time will
+        // have the same set of application local directories.
+        localDirs.foreach { rootDir =>
+          try {
+            val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+            // Only one container will create this directory. The filesystem will handle any race
+            // conditions.
+            if (!mergeDir.exists()) {
+              Utils.createDirWith770(mergeDir)
+              for (dirNum <- 0 until subDirsPerLocalDir) {
+                val sudDir = new File(mergeDir, "%02x".format(dirNum))
+                Utils.createDirWith770(sudDir)
+              }
+            }
+            logInfo(s"Merge directory at $mergeDir")
+          } catch {
+            case e: IOException =>
+              logError(
+                s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+          }
+        }
+      }
+    }
+    findActiveMergedShuffleDirs(conf)

Review comment:
       I think we can do:
   
   ```scala
   If (Utils.isPushBasedShuffleEnabled(conf)) {
    ....
   } else {
    Array.empty[File]
   }
   ```
   
   we don't need the `Option` here.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +729,24 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId)
+  }
+
+  /**
+   * Get the local merged shuffle block metada data for the given block ID.
+   */
+  def getMergedBlockMeta(blockId: ShuffleBlockId): MergedBlockMeta = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId)
+  }
+
+

Review comment:
       nit: redundant blank line.

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -1448,6 +1445,17 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
     conf.set(SHUFFLE_SERVICE_ENABLED, true)
     assert(Utils.isPushBasedShuffleEnabled(conf) === true)
   }
+
+  test("Test create dir with 770") {
+    val testDir = new File("target/testDir");
+    FileUtils.deleteQuietly(testDir)
+    Utils.createDirWith770(testDir)

Review comment:
       Shall we check the permission of the created dir?

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -30,14 +29,12 @@ import java.util.zip.GZIPOutputStream
 
 import scala.collection.mutable.ListBuffer
 import scala.util.Random
-
 import com.google.common.io.Files
-import org.apache.commons.io.IOUtils
+import org.apache.commons.io.{FileUtils, IOUtils}
 import org.apache.commons.lang3.{JavaVersion, SystemUtils}
 import org.apache.commons.math3.stat.inference.ChiSquareTest
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
-

Review comment:
       ditto

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }
+      if (!mergeDirCreated) {
+        // This executor didn't see any merge_manager directories, it will start creating them.
+        // It's possible that the other executors launched at the same time may also reach here but
+        // we are working on the assumption that the executors launched around the same time will
+        // have the same set of application local directories.
+        localDirs.foreach { rootDir =>
+          try {
+            val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+            // Only one container will create this directory. The filesystem will handle any race
+            // conditions.
+            if (!mergeDir.exists()) {
+              Utils.createDirWith770(mergeDir)
+              for (dirNum <- 0 until subDirsPerLocalDir) {
+                val sudDir = new File(mergeDir, "%02x".format(dirNum))
+                Utils.createDirWith770(sudDir)
+              }
+            }
+            logInfo(s"Merge directory at $mergeDir")
+          } catch {
+            case e: IOException =>
+              logError(
+                s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+          }
+        }
+      }
+    }
+    findActiveMergedShuffleDirs(conf)
+  }
+
+  private def findActiveMergedShuffleDirs(conf: SparkConf): Option[Array[File]] = {

Review comment:
       Shall inline this function? It's only called once.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }

Review comment:
       Is this necessary?  If you want to skip creating the directory below I think you can do:
   
   ```scala
   locaDirs.forall {  rootDir =>
    val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
    if (mergeDir.exist()) {
     false
    } else {
     Utils.createDirWith770(mergeDir)
     for (dirNum <- 0 until subDirsPerLocalDir) {
      val sudDir = new File(mergeDir, "%02x".format(dirNum))
      Utils.createDirWith770(sudDir)
     }
     true
   }

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,37 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {

Review comment:
       Shall we add the permission as a parameter and make this function more general considering it's a util function?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +91,34 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.appId, mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.appId, mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.appId, mergedMetaBlockId.name)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got ${blockId}")
+    }
+  }
+
+  private def getMergedShuffleFile(appId: String, filename: String): File = {

Review comment:
       `appId` not used?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +91,34 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile

Review comment:
       nit: we warp this with `[[...]`, so the class can be linked.

##########
File path: core/src/main/scala/org/apache/spark/network/BlockDataManager.scala
##########
@@ -71,4 +71,9 @@ trait BlockDataManager {
    * Release locks acquired by [[putBlockData()]] and [[getLocalBlockData()]].
    */
   def releaseLock(blockId: BlockId, taskContext: Option[TaskContext]): Unit
+
+  /**
+   * Get the local merged shuffle block data
+   */
+  def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer]

Review comment:
       Shall we rename it to `getLocalMergedBlockData` if it's local only?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)

Review comment:
       Shall we make this a class field so we don't call it multiple times? (e.g., `createLocalDirs`, `findActiveMergedShuffleDirs` also call it) 

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"

Review comment:
       Shall we use `shuffleMerged` instead? It's more consistent with others, e.g., `shufflePush`. And probably be useful somewhere when we need to filter the shuffle-related block by checking the prefix "shuffle".

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +358,48 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val offsets = Utils.tryWithResource {
+      new DataInputStream(Files.newInputStream(indexFile.toPath))
+    } { dis =>
+      val buffer = ByteBuffer.allocate(size)
+      dis.readFully(buffer.array)
+      buffer.asLongBuffer
+    }
+    // Number of chunks is number of indexes - 1
+    val numChunks = size / 8 - 1
+    val chunkSizes = new Array[Long](numChunks)
+    for (index <- 0 until numChunks) {
+      chunkSizes(index) = offsets.get(index + 1) - offsets.get(index)
+    }
+    chunkSizes.indices.map {
+      index =>
+        new FileSegmentManagedBuffer(transportConf, dataFile,
+          offsets.get(index), chunkSizes(index))
+    }

Review comment:
       nit:
   
   ```suggestion
       for (index <- 0 until numChunks) {
         new FileSegmentManagedBuffer(transportConf, dataFile, 
          offsets.get(index)
          offsets.get(index + 1) - offsets.get(index)
       }
   ```

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +91,34 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * org.apache.spark.network.shuffle.RemoteBlockPushResolver#getMergedShuffleFile
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.appId, mergedBlockId.name)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.appId, mergedIndexBlockId.name)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.appId, mergedMetaBlockId.name)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got ${blockId}")

Review comment:
       nit: unnecessary "{}"

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -30,14 +29,12 @@ import java.util.zip.GZIPOutputStream
 
 import scala.collection.mutable.ListBuffer
 import scala.util.Random
-

Review comment:
       nit: revert the blank.

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,8 +17,7 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File, FileOutputStream, PrintStream, SequenceInputStream}

Review comment:
       nit: `import java.io._`




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640855529



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -824,17 +919,17 @@ public void testFailureAfterDuplicateBlockDoesNotInterfereActiveStream() throws
   private void useTestFiles(boolean useTestIndexFile, boolean useTestMetaFile) throws IOException {
     pushResolver = new RemoteBlockPushResolver(conf) {
       @Override
-      AppShufflePartitionInfo newAppShufflePartitionInfo(AppShuffleId appShuffleId, int reduceId,
-        File dataFile, File indexFile, File metaFile) throws IOException {
+      AppShufflePartitionInfo newAppShufflePartitionInfo(AppAttemptShuffleId appAttemptShuffleId, int reduceId,
+                                                         File dataFile, File indexFile, File metaFile) throws IOException {

Review comment:
       Ni: indentation

##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -61,6 +61,17 @@
 
   private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
   private final String TEST_APP = "testApp";
+  private final String MERGE_DIRECTORY = "merge_directory";
+  private final int NON_ATTEMPTID = -1;

Review comment:
       NIt: NO_ATTEMPT_ID

##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -149,10 +160,10 @@ public void testFinalizeWithMultipleReducePartitions() throws IOException {
   @Test
   public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
     StreamCallbackWithID stream1 =
-      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, -1, 0, 0, 0, 0));

Review comment:
       NIT: Use NO_ATTEMPT_ID here and at other places

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,64 +123,85 @@ public ShuffleIndexInformation load(File file) throws IOException {
    * shuffle does not exist, initializes the metadata.
    */
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
-      AppShuffleId appShuffleId,
+      AppAttemptShuffleId appAttemptShuffleId,
+      AppAttemptPathsInfo appAttemptPathsInfo,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
-    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+    File dataFile = getMergedShuffleDataFile(
+      appAttemptPathsInfo, appAttemptShuffleId.appId, appAttemptShuffleId.shuffleId, reduceId);
+    if (!partitions.containsKey(appAttemptShuffleId) && dataFile.exists()) {
       // If this partition is already finalized then the partitions map will not contain
       // the appShuffleId but the data file would exist. In that case the block is considered late.
       return null;
     }
-    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
-      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
-    return shufflePartitions.computeIfAbsent(reduceId, key -> {
-      // It only gets here when the key is not present in the map. This could either
-      // be the first time the merge manager receives a pushed block for a given application
-      // shuffle partition, or after the merged shuffle file is finalized. We handle these
-      // two cases accordingly by checking if the file already exists.
-      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
-      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
-      try {
-        if (dataFile.exists()) {
-          return null;
-        } else {
-          return newAppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
-        }
-      } catch (IOException e) {
-        logger.error(
-          "Cannot create merged shuffle partition with data file {}, index file {}, and "
-            + "meta file {}", dataFile.getAbsolutePath(),
-            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
-        throw new RuntimeException(
-          String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s "
-          + "reduceId %s", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+    // While processing the application remove, where the shuffle partitions info for the specific
+    // application will be cleaned up, this method will still be called to create new partitions
+    // as of receiving the push blocks. To avoid the potential memory leak, before creating the
+    // empty hashmap for storing the shuffle partitions information in the partitions hashmap,
+    // we need to make sure that the entry for the specific application must still exist in
+    // appAttemptsPathInfo hashmap. Otherwise, the push blocks should be ignored.

Review comment:
       I don't see how this completely avoids the issue. There can be a context switch after line 150 and another thread can call `applicationRemoved`. I don't think we should try to solve this issue with this PR. This is a problem with existing code so why not create another jira for it. The goal of this jira is to add support in DiskBlockManager to server local merged data and we keep deviating from it.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +343,63 @@ void deleteExecutorDirs(Path[] dirs) {
     }
   }
 
+  /**
+   * Create StreamCallback for invalid push blocks with the specific error message.
+   * If specific error message is null, this StreamCallback won't throw exception in client.
+   */
+  private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+      String streamId,
+      String errorMessage) {
+    return new StreamCallbackWithID() {
+      @Override
+      public String getID() {
+        return streamId;
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) {
+        // Ignore the requests. It reaches here either when a request is received after the
+        // shuffle file is finalized or when a request is for a duplicate block.
+      }
+
+      @Override
+      public void onComplete(String streamId) {
+        if (errorMessage != null) {
+          // Throw an exception here so the block data is drained from channel and server
+          // responds RpcFailure to the client.
+          throw new RuntimeException(String.format("Block %s %s", streamId, errorMessage));
+        }
+        // For duplicate block that is received before the shuffle merge finalizes, the
+        // server should respond success to the client.
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable cause) {
+      }
+    };
+  }
+
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppAttemptPathsInfo appAttemptPathsInfo = getAppAttemptPathsInfo(msg.appId);
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId =
+      new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+      // If this Block belongs to a former application attempt, it is considered late,
+      // as only the blocks from the current application attempt will be merged
+      return createCallbackForInvalidPushBlocks(streamId,
+        ErrorHandler.BlockPushErrorHandler.NEWER_ATTEMPT_HAS_STARTED_MESSAGE_SUFFIX);

Review comment:
       Please add a TODO  as well as jira ticket for this that the client need to handle this failure. 

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +53,14 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  createLocalDirsForMergedShuffleBlocks(conf)
+
+  private[spark] lazy val activeMergedShuffleDirs: Option[Array[File]] =
+    findActiveMergedShuffleDirs(conf)
+

Review comment:
       If this executor is not creating the merged directory, then currently it seems that activeMergedShuffleDirs will be empty. Check line 216. 

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
##########
@@ -35,7 +35,11 @@
   public final String[] localDirs;
   /** Number of subdirectories created within each localDir. */
   public final int subDirsPerLocalDir;
-  /** Shuffle manager (SortShuffleManager) that the executor is using. */
+  /** Shuffle manager (SortShuffleManager) that the executor is using.

Review comment:
       Nit: Since it is no longer one line, start it from the next line. Also fix line 42

##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -61,6 +61,17 @@
 
   private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
   private final String TEST_APP = "testApp";
+  private final String MERGE_DIRECTORY = "merge_directory";
+  private final int NON_ATTEMPTID = -1;
+  private final int ATTEMPTID_1 = 1;

Review comment:
       Nit: ATTEMPT_ID_1

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -336,51 +436,26 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
       && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
         : partitionInfoBeforeCheck;
-    final String streamId = String.format("%s_%d_%d_%d",
-      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
-      msg.reduceId);
     if (partitionInfo != null) {
       return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
     } else {
       // For a duplicate block or a block which is late, respond back with a callback that handles
       // them differently.
-      return new StreamCallbackWithID() {
-        @Override
-        public String getID() {
-          return streamId;
-        }
-
-        @Override
-        public void onData(String streamId, ByteBuffer buf) {
-          // Ignore the requests. It reaches here either when a request is received after the
-          // shuffle file is finalized or when a request is for a duplicate block.
-        }
-
-        @Override
-        public void onComplete(String streamId) {
-          if (isTooLate) {
-            // Throw an exception here so the block data is drained from channel and server
-            // responds RpcFailure to the client.
-            throw new RuntimeException(String.format("Block %s %s", streamId,
-              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
-          }
-          // For duplicate block that is received before the shuffle merge finalizes, the
-          // server should respond success to the client.
-        }
-
-        @Override
-        public void onFailure(String streamId, Throwable cause) {
-        }
-      };
+      if (isTooLate) {

Review comment:
       Can you please explain why you have broken down this into if/else here? What will will the client do when it receives a `null` as a failure message? 

##########
File path: core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockResolver.scala
##########
@@ -40,5 +41,15 @@ trait ShuffleBlockResolver {
    */
   def getBlockData(blockId: BlockId, dirs: Option[Array[String]] = None): ManagedBuffer
 
+  /**
+   * Retrieve the data for the specified merged shuffle block as multiple chunks.
+   */
+  def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer]

Review comment:
       All of these are missing the `dirs` parameter

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +53,14 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  createLocalDirsForMergedShuffleBlocks(conf)
+
+  private[spark] lazy val activeMergedShuffleDirs: Option[Array[File]] =
+    findActiveMergedShuffleDirs(conf)
+

Review comment:
       BTW this is no longer needed. The dirs are passed to the getMerged... APIs.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,23 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  override def getLocalMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {

Review comment:
       This is also the older API. @zhouyejoe We need to add the dirs. Please look at these APIs in https://github.com/apache/spark/pull/32140/ 
   

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +101,34 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(java.lang.String,
+   *   java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId): File = {

Review comment:
       @zhouyejoe This is outdated code. We need to pass the `dirs: Option[Array[String]]`to this method to support discovering location of merged block directories when dynamic resource allocation is enabled.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +490,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == ATTEMPT_ID_UNDEFINED) {
+          // When attemptId is -1, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appAttemptsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt wil register
+          // the merge dirs in External Shuffle Service. Any later ExecutorRegister message
+          // from the same application attempt will not override the merge dirs. But it can
+          // be overridden by ExecutorRegister message from newer application attempt,
+          // and former attempts' shuffle partitions information will also be cleaned up.
+          AtomicBoolean newAttemptRegistered = new AtomicBoolean(false);
+          appAttemptsPathsInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo == null || (appAttemptPathsInfo != null
+                && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId)) {
+              appAttemptPathsInfo =
+                new AppAttemptPathsInfo(
+                  appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+                  mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+              newAttemptRegistered.set(true);
+            }
+            return appAttemptPathsInfo;
+          });
+          if (newAttemptRegistered.get()) {
+            logger.warn("Remove shuffle partition info for {}_{} generated for former attempts",
+              appId, mergeDirectoryMeta.attemptId);
+            cleanupShufflePartitionInfo(appId, mergeDirectoryMeta.attemptId);

Review comment:
       `cleanupShufflePartitionInfo` iterates over the whole `partitions` map so we have increased the latency of registration. This cleanup can happen asynchronously. Please add a TODO and create a jira for it as well.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648874848



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"$MERGE_MANAGER_DIR dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {

Review comment:
       Updated the method name and revert back to only mkdir with permission 770.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638106460



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -761,28 +818,31 @@ public boolean equals(Object o) {
       if (o == null || getClass() != o.getClass()) {
         return false;
       }
-      AppShuffleId that = (AppShuffleId) o;
-      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+      AppAttemptShuffleId that = (AppAttemptShuffleId) o;
+      return Objects.equal(appId, that.appId)
+        && attemptId == that.attemptId
+        && shuffleId == that.shuffleId;
     }
 
     @Override
     public int hashCode() {
-      return Objects.hashCode(appId, shuffleId);
+      return Objects.hashCode(appId, attemptId, shuffleId);
     }
 
     @Override
     public String toString() {
-      return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE)
-        .append("appId", appId)
-        .append("shuffleId", shuffleId)
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("attemptId", attemptId)
+        .add("shuffleId", shuffleId)
         .toString();
     }
   }
 
   /** Metadata tracked for an actively merged shuffle partition */
-  public static class AppShufflePartitionInfo {
+  public static class AppAttemptShufflePartitionInfo {

Review comment:
       We only create a partition info for the latest attempt. I don't really think renaming this class is necessary. 
   The java doc also says this. Here `attempt` has no significance since it is just the part of the `AppAttemptShuffleId` which is already renamed. This just makes the class name longer and adds bunch of other changes which just because of this change in name




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644962656



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       On second thought, given `Utils` is not available, there might not be an alternative location for this.
   We might need to look at a common infra module for utilities like this which all modules can depend on (or is there something already ?) ... thoughts @Ngone51 ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648745059



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }

Review comment:
       Updated the code as described above. So there is no more need to forloop and check if any dir contains the merge_dir.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-820954486


   Thanks for reviewing the PR. I am addressing the comments, will update the PR soon.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-849920007


   @zhouyejoe @mridulm 
   I agree with @otterc that it would be better to deal with the issue with the race conditions in a separate jira.
   I think we have already seen several potential symptoms of this issue during corner cases, i.e. partition info object not properly cleaned after a new app attempt or even after application finishes.
   The underlying issue behind all of these is that the code currently does not properly deal with concurrency controls of access to the `partitions` and `appPathsInfo` concurrent hash maps.
   This is an existing problem, although exacerbated by the introduction of the handling of multiple app attempts.
   I think creating a separate ticket to fix this underlying issue instead of fixing the different symptoms of this issue in this PR would be better.
   This way we can come up with a more systematic way to resolve the issue without over bloating this PR with changes not related to its original scope.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r633744774



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,7 +116,7 @@ public ShuffleIndexInformation load(File file) throws IOException {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleId appShuffleId,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId.appId, appShuffleId.shuffleId, reduceId);

Review comment:
       When the server receives a pushBlock message then there are 2 cases:
   1. It belongs to previous attempt: in this case it needs to ignore that message. If we don't then it will corrupt the file here.
   2. It belongs to latest attempt: in this case we should continue merging
   
   Need to handle case (1)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-858883870


   **[Test build #139658 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139658/testReport)** for PR 32007 at commit [`e630725`](https://github.com/apache/spark/commit/e630725ca5c161cea62a2afcc7668a67a3e6d72e).


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637965615



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +59,21 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  /**
+   * Create merge directories
+   */
+  private val mergeDirName: String = {
+    try {
+      val attemptId = conf.get("spark.app.attempt.id")
+      MERGE_DIRECTORY + "_" + Utils.sanitizeDirName(attemptId)
+    } catch {
+      case _: NoSuchElementException =>
+        MERGE_DIRECTORY
+    }
+  }

Review comment:
       nit: leave a blank line below




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644914904



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       I'm fine with it.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644520610



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def geLocalMergedBlockMeta(

Review comment:
       Added Local here, as suggested by @Ngone51. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859044030


   **[Test build #139658 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/139658/testReport)** for PR 32007 at commit [`e630725`](https://github.com/apache/spark/commit/e630725ca5c161cea62a2afcc7668a67a3e6d72e).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r619966403



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To minimize change, why not get the executor to create the directory as "merge_directory_<attempt_id>" ? (if missing)
   While registering, ESS can pick up the highest attempt id and associate the executor with that attempt id (and so merge_directory).
   
   Thoughts ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638403604



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -419,4 +419,8 @@ public long mergedIndexCacheSize() {
   public int ioExceptionsThresholdDuringMerge() {
     return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
   }
+
+  public int appAttemptId() {
+    return conf.getInt("spark.app.attempt.id", -1);

Review comment:
       Oh..I mean the latter one.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644423043



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +349,64 @@ void deleteExecutorDirs(Path[] dirs) {
     }
   }
 
+  /**
+   * Create StreamCallback for invalid push blocks with the specific error message.
+   * If specific error message is null, this StreamCallback won't throw exception in client.
+   */
+  private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+      String streamId,
+      String errorMessage) {
+    return new StreamCallbackWithID() {
+      @Override
+      public String getID() {
+        return streamId;
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) {
+        // Ignore the requests. It reaches here either when a request is received after the
+        // shuffle file is finalized or when a request is for a duplicate block.
+      }
+
+      @Override
+      public void onComplete(String streamId) {
+        if (errorMessage != null) {
+          // Throw an exception here so the block data is drained from channel and server
+          // responds RpcFailure to the client.
+          throw new RuntimeException(String.format("Block %s %s", streamId, errorMessage));
+        }
+        // For duplicate block that is received before the shuffle merge finalizes, the
+        // server should respond success to the client.
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable cause) {
+      }
+    };
+  }
+
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppAttemptPathsInfo appAttemptPathsInfo = getAppAttemptPathsInfo(msg.appId);
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId =
+      new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+      // If this Block belongs to a former application attempt, it is considered late,
+      // as only the blocks from the current application attempt will be merged
+      // TODO: [SPARK-35548] Client should be updated to handle this error.
+      return createCallbackForInvalidPushBlocks(streamId,

Review comment:
       Removed from the slim down PR.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }

Review comment:
       The original logic:
   For loop each local dir, if there is any merge_dir created by other executors, this executor will not create any local dirs.
   Potential issue:
   Suppose Executor 1 created the merge_dirs under /tmp/[a-c]. Executor2 launched slightly later, it got local dirs /tmp/[b-d]. Executor2 would not create merge_dirs in any of them as it found that another executor has created the merge dir in /tmp/b. But if the executor registration message from Executor2 gets handled prior to Executor1's, shuffle service will use the /tmp/[b-d] as the merge dirs. However, the merge_dir under /tmp/d has not been created by Executor2.
   
   The update logic:
   Every Executor should try to create the merge_dir under the local dirs, which makes sure no matter what ExecutorRegister message gets received in shuffle service, it is guaranteed that the merge_dir is there with permission 770.
   @otterc
   
   

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       Fixed and added a unit test in HostLocalShuffleReadingSuite to check whether the hostLocalDirManager gets initiated when push based shuffle is enabled.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       IndexShuffleBlockResolver will always fetch a local disk, but in ShuffleBlockResolver, there is no "Local" added in the method name, similarly to original shuffle fetch. In ShuffleBlockResolver and IndexShuffleBlockResolver, "Local" is not added to the method name. But in BlockManager, we have getLocalBlockData and getHostLocalShuffleData.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       Similarly to original shuffle fetch, in ShuffleBlockResolver and IndexShuffleBlockResolver, "Local" is not added to the method name. But in BlockManager, we have getLocalBlockData and getHostLocalShuffleData.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def geLocalMergedBlockMeta(

Review comment:
       Added Local here, as suggested by @Ngone51. 

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -129,6 +155,9 @@ object BlockId {
   val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r
   val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r
   val SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r
+  val SHUFFLE_MERGED = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).data".r
+  val SHUFFLE_MERGED_INDEX = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).index".r
+  val SHUFFLE_MERGED_META = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).meta".r

Review comment:
       Just figured it out, this is for unit testing, similarly to other unit tests for case classes defined in BlockId.scala

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       Updated as @mridulm suggested.

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       Moved to DiskBlockManager. Also need to change the access modifiers for MAX_DIR_CREATION_ATTEMPTS in Utils.scala.

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       You did recommend this change in former reviews. https://github.com/apache/spark/pull/32007#discussion_r622840432

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+      val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+      sparkMaxAttempts match {
+        case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+        case None => yarnMaxAttempts
+      }

Review comment:
       The code does exist in Spark code base, but in the module of resource-manager/yarn. Should we move this part to Utils.scala and then call it in YarnRMClient? Or we can just make a code duplicate here?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638032793



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.

Review comment:
       This confused me at first glance. I think you mean "when attemptId is -1, it means the application doesn't have the attemptId." Could you reword it?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642024816



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -290,16 +349,64 @@ void deleteExecutorDirs(Path[] dirs) {
     }
   }
 
+  /**
+   * Create StreamCallback for invalid push blocks with the specific error message.
+   * If specific error message is null, this StreamCallback won't throw exception in client.
+   */
+  private StreamCallbackWithID createCallbackForInvalidPushBlocks(
+      String streamId,
+      String errorMessage) {
+    return new StreamCallbackWithID() {
+      @Override
+      public String getID() {
+        return streamId;
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) {
+        // Ignore the requests. It reaches here either when a request is received after the
+        // shuffle file is finalized or when a request is for a duplicate block.
+      }
+
+      @Override
+      public void onComplete(String streamId) {
+        if (errorMessage != null) {
+          // Throw an exception here so the block data is drained from channel and server
+          // responds RpcFailure to the client.
+          throw new RuntimeException(String.format("Block %s %s", streamId, errorMessage));
+        }
+        // For duplicate block that is received before the shuffle merge finalizes, the
+        // server should respond success to the client.
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable cause) {
+      }
+    };
+  }
+
   @Override
   public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
     // Retrieve merged shuffle file metadata
-    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppAttemptPathsInfo appAttemptPathsInfo = getAppAttemptPathsInfo(msg.appId);
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, msg.shuffleId, msg.mapIndex, msg.reduceId);
+    AppAttemptShuffleId appAttemptShuffleId =
+      new AppAttemptShuffleId(msg.appId, msg.attemptId, msg.shuffleId);
+    if (appAttemptPathsInfo.attemptId != appAttemptShuffleId.attemptId) {
+      // If this Block belongs to a former application attempt, it is considered late,
+      // as only the blocks from the current application attempt will be merged
+      // TODO: [SPARK-35548] Client should be updated to handle this error.
+      return createCallbackForInvalidPushBlocks(streamId,

Review comment:
       Is there a need to create a valid `StreamCallback` in this case? Can we just not throw the RuntimeException with the `NEWER_ATTEMPT_HAS_STARTED_MESSAGE_SUFFIX` when it tries to create the stream? This will also make the refactoring  below which added `createCallbackForInvalidPushBlocks` unnecessary. 
   We do throw RuntimeException when `Cannot initialize merged shuffle partition for appId %s shuffleId %s"reduceId %s` from `getOrCreateAppShufflePartitionInfo`, so we can immediately fail to create the stream here as well.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644519782



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       IndexShuffleBlockResolver will always fetch a local disk, but in ShuffleBlockResolver, there is no "Local" added in the method name, similarly to original shuffle fetch. In ShuffleBlockResolver and IndexShuffleBlockResolver, "Local" is not added to the method name. But in BlockManager, we have getLocalBlockData and getHostLocalShuffleData.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644973318



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       Yeah you are right Mridul. It should be 
   ```
   if ((conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) ||
             Utils.isPushBasedShuffleEnabled(conf))
   ```




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637355653



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,7 +116,7 @@ public ShuffleIndexInformation load(File file) throws IOException {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleId appShuffleId,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId.appId, appShuffleId.shuffleId, reduceId);

Review comment:
       Added AttemptID check here. If the attemptID is different with the one recorded in AppAttemptPathInfo, it will return null and will take this block as a late block, so it will be ignored.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -991,22 +1023,26 @@ int getNumIOExceptions() {
   /**
    * Wraps all the information related to the merge directory of an application.
    */
-  private static class AppPathsInfo {
+  private static class AppAttemptPathsInfo {
 
+    private final int attemptId;
     private final String[] activeLocalDirs;
     private final int subDirsPerLocalDir;
 
-    private AppPathsInfo(
+    private AppAttemptPathsInfo(
         String appId,
+        int attemptId,

Review comment:
       Changed to AppAttemptShuffleId




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615220493



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +193,58 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by external
+   * shuffle services if push based shuffle is enabled. Note that the files in this directory
+   * will be created by the external shuffle services. We only create the merge_manager directories
+   * here because currently the shuffle service doesn't have permission to create directories
+   * under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {

Review comment:
       Added subdirectories count check in UT




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-859033409


   
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/44186/
   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r604559548



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi

Review comment:
       Need to add `@Since("3.2.0")`




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r623036330



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       I have a very tricky idea here, which is based on solution 3: 
   
   When an executor of an application attempt X tries to create the merge directory, it could first check whether the dir `merge_manager_X-1` exists. If exists, delete it. And if dir `merge_manager_X` not exists, then create `merge_manager_X`. And if the executor becomes the one who creates the merge dir, we send the `ExecutorShuffleInfo` with the special `shuffleManager`, e.g., "sort_merge_manager_attemptX". And `ExternalBlockHandler` can parse the `shuffleManager` into two parts. So `ExternalShuffleBlockResolver` can still register the normal `ExecutorShuffleInfo` and  `RemoteBlockPushResolver` can know whether to update the merge dir.
   
   
   Besides, I'd prefer soultion1. I think it's reasonable to add a new message for the push-based shuffle. We can have a new type for it, e.g.,`RegisterMergeDirectory`, which includes the merge directory directly and attempted too (of course). 
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r647785023



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -149,6 +178,12 @@ object BlockId {
       ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt)
     case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) =>
       ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED(appId, shuffleId, reduceId) =>

Review comment:
       The conversation was marked resolved .. Did we make this change and then revert it back ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638422058



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       There is a race between this cleanup and some executor adding an entry into partitions map.
   In a nutshell, we need to maintain an `applications` `Set` which contains currently running apps - and all updates/checks on this Set should be via `synchronized` block.
   a) In `registerExecutor`, add to this `Set`
   b) In `applicationRemoved`, remove from this `Set` before doing `cleanupShufflePartitionInfo`.
   c) In `getOrCreateAppShufflePartitionInfo`, in `partitions.computeIfAbsent`, lock `applications` and check if incoming request application  is in this `Set`, and if missing return `null` (and handle a `null` value for `shufflePartitions`).
   
   This handles the race condition of `getOrCreateAppShufflePartitionInfo` (from a remote executor) overlapping with `applicationRemoved` (on local NM).
   This also means the race condition @otterc referenced [above](https://github.com/apache/spark/pull/32007/files#r638325094) will not occur (application removal will ensure all records are cleaned up - so stale entries might live utmost until application termination - not after).
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637971817



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.

Review comment:
       If that's the case, shouldn't we only create the merge_manager directory after checking all the `configuredLocalDirs`?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r625477570



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       > If we are moving the delete to the executor, then we could formulate it to make the change minimal right ?
   
   Yes, and thanks for bringing the detailed steps, which looks good.
   
   
   BTW, there's another tricky way to ship the merge directory that is appending the merge directory into `ExecutorShuffleInfo.localDirs`.
   
   Also, adding a new RPC doesn't seem to help a lot here according to the discussion.
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r634860230



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java
##########
@@ -30,12 +30,15 @@
  */
 public class FinalizeShuffleMerge extends BlockTransferMessage {
   public final String appId;
+  public final int attemptId;

Review comment:
       In rare cases, the driver container from the last attempt may still be running in the cluster whereas the driver from the second attempt has been launched. It is better that we can also have the attemptId in the FinalizeShuffleMessage.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r618802072



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Thanks for the review and discussion on this issue. We have reached an agreement that the current implementation will cause issues when the application has multiple attempts. 
   **Issue description:**
   Currently the merge dir is created under the application local tmp dir by the executors. For example, for application_12345, executor1 running on Node1 will create the merge_dir under /[yarn-local-dirs]/usercache/[username]/appcache/application_12345/merge_dir. Even though there are other executors from this application running on Node1, they will not create those dirs as it has been created. When executor1 register itself with the shuffle service on Node1, it will also register the local dirs and let shuffle service know where the unmerged shuffle data is stored. So in this case, it will register  /[yarn-local-dirs]/usercache/[username]/appcache/application_12345/blockmgr_RandomID. ESS will use this dirs and assume the merge dirs have been created under /[yarn-local-dirs]/usercache/[username]/appcache/application_12345/merge_dir. For the executor register messages from the same application, the first message will register the merge dirs in shuffle service, others will be ignored.
  While the application finishes, we rely on Yarn NodeManager to clean up the whole dir /[yarn-local-dirs]/usercache/[username]/appcache/application_12345. 
   1nd issue: Since shuffle service will only use the first executor register message from the same application, the original dirs list from the first executor in the first attempt will be kept all the way during the later attempts run. 
   2st issue: While in the case of multiple attempts from the same application, this dir won’t get cleaned up. So the original merged shuffle files generated by shuffle service from the former attempts will still exist within the merge_dir. This would be stale data but shuffle service will use these stale data to serve the merge shuffle fetch in later attempts.
   
   **Root cause of the issue:**
   For original unmerged shuffle files, there won’t be such issues as 1. Every executor will register its own local dirs to shuffle service 2. Block manager dirs are encoded with randomIDs on the path. For merged shuffle dir, we only use the first ever register executor message to get the merge_dir base location and ignore all the later ones. But in executor register message, there is no attemptID encoded, so shuffle service cannot get to know whether they are from different attempts. Yarn will not cleanup tmp directories while the application is still running, even though there are multiple attempts.
   
   **Proposed solution:**
   In either proposed solution, we have to get the attemptID encoded into the executor register protocol. After we have the attemptID encoded and shuffle service can use this attemptID as the way to distinguish among multiple attempts.
   Solution 1: We can create a new protocol for RegisterExecutor, which adds a new fields: AttemptID, compared with the original RegisterExecutor protocol. This new protocol will be used when push based shuffle is enabled.
   Solution 2: Don’t create new protocol and just leverage the existing protocol. We can encode “appID” + “attemptID” as the “new appID” in the RegisterExecutor, let’s call it appAttemptID. Shuffle service will be able to tell whether the String is the original appID or appAttemptID with Regex, and then deal with it.
   
   CC @Ngone51 @tgravescs @attilapiros. Can you share your opinions over the issues and proposed solutions? Thanks.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r624885672



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       > If ESS does not support the new RPC, how is the spark application supposed to behave ?
   
   So this is the case that the cluster does not support push-based shuffle but client is sending push-based shuffle related messages. In this case, it should be okay that the executor fails early because it is trying push-based shuffle which will not work. For this case, we can throw a `SparkException` with message that "push-based is not supported by the cluster so please turn it off".
   
   > If we are taking this path, it would be better for ESS to manage the merger location entirely - and not have executors create/update it (as discussed above). It will help ESS evolve independently.
   
   ESS cannot create the merge_manager directory under application local directory because it doesn't have permissions to do so. App local dirs have permission `750` and shuffle service is part of NM process that are run usually by `yarn` user. 
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614501382



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))

Review comment:
       Do we need to update here? Similar to the changes in SPARK-21475?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638417136



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -34,15 +32,17 @@
  */
 public class PushBlockStream extends BlockTransferMessage {
   public final String appId;
+  public final int attemptId;
   public final int shuffleId;
   public final int mapIndex;
   public final int reduceId;
   // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of
   // blocks to be pushed.
   public final int index;
 
-  public PushBlockStream(String appId, int shuffleId, int mapIndex, int reduceId, int index) {
+  public PushBlockStream(String appId, int attemptId, int shuffleId, int mapIndex, int reduceId, int index) {

Review comment:
       Nit: line too long, break into 2 lines.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r626212871



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       > Also, adding a new RPC doesn't seem to help a lot here according to the discussion.
   
   I agree, minimizing change to protocol should be better here.
   Let us try without introducing RPC - it does not allow for clean separation of concerns anyway (based on @otterc's clarification above) ... so making it entirely managed at executors would be better.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638426369



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,32 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+    appId: String,
+    shuffleId: Int,
+    reduceId: Int) extends BlockId {
+  override def name: String =
+    "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".index"
+}
+
+@Since("3.2.0")
+@DeveloperApi
+case class ShuffleMergedMetaBlockId(
+    appId: String,
+    shuffleId: Int,
+    reduceId: Int) extends BlockId {
+  override def name: String =
+    "shuffleMerged_" + appId + "_" + shuffleId + "_" + reduceId + ".meta"

Review comment:
       I see @zhouyejoe made the change on server side to create files with prefix `shuffleMerged` as well.
   However, I still feel that `mergedShuffle` reads better and is in line with how we named these files in `RemoteBlockPusherResolver#getMergedShuffleXXXFile`.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648817463



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"$MERGE_MANAGER_DIR dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {

Review comment:
       I'm ok with it. And I think we need a more specific function name after taking away the `permission` parameter.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r633726943



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -744,10 +771,12 @@ AppShufflePartitionInfo getPartitionInfo() {
    */
   public static class AppShuffleId {
     public final String appId;
+    public final int attemptId;
     public final int shuffleId;
 
-    AppShuffleId(String appId, int shuffleId) {
+    AppShuffleId(String appId, int attemptId, int shuffleId) {

Review comment:
       I think we don't need to change this instead we can maintain another map with appId and latest attempt information. It would be best if we keep only latest partition infos in the `partitions` map and remove any entries with old attempts.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FinalizeShuffleMerge.java
##########
@@ -30,12 +30,15 @@
  */
 public class FinalizeShuffleMerge extends BlockTransferMessage {
   public final String appId;
+  public final int attemptId;

Review comment:
       FinalizeShuffleMessage is sent from the driver to the shuffle services. Is it possible for the driver of a previous attempt to be alive when the driver of a new attempt is running? 
   I think whenever the server receives `FinalizeShuffeMerge` it should always assume it is from the latest attempt

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -117,7 +116,7 @@ public ShuffleIndexInformation load(File file) throws IOException {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleId appShuffleId,
       int reduceId) {
-    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId.appId, appShuffleId.shuffleId, reduceId);

Review comment:
       When the server receives a pushBlock message then there are 2 cases:
   1. It belongs to previous attempt: in this case it needs to ignore that message. If we don't then it will corrupt the file here.
   2. It belongs to latest attempt: in this case we should continue merging

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -414,23 +411,53 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         Longs.toArray(sizes));
     }
     partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == 0) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not 0, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // won't override the merge dirs. But it can be overridden by ExecutorRegister
+          // message from new app attempts.
+          appsPathInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId) {
+              appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
+                executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+            }
+            return appAttemptPathsInfo;

Review comment:
       We are not removing entries from `partitions` here? 
   If we don't maintain just active partitionsInfo in the `partitions` map then it will add complexity while saving in level db. Also these entries are not getting removed in the finalization so that just means it means that they will still in memory until the app is removed but they don't have any use.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -991,22 +1023,26 @@ int getNumIOExceptions() {
   /**
    * Wraps all the information related to the merge directory of an application.
    */
-  private static class AppPathsInfo {
+  private static class AppAttemptPathsInfo {
 
+    private final int attemptId;
     private final String[] activeLocalDirs;
     private final int subDirsPerLocalDir;
 
-    private AppPathsInfo(
+    private AppAttemptPathsInfo(
         String appId,
+        int attemptId,

Review comment:
       We can avoid this by maintaining another map maintaining latest attempts of all apps. Or if you want to include `attemptId` in this class then maybe change the name of this class to `AppMergedMeta` or likewise and modify the description 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644689361



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {

Review comment:
       nit: "createMergeManagerDirIfNeeded()" ?
   
   btw: `conf` can be omitted as it's a member field.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to

Review comment:
       nit: `shuffle service` -> `external shuffle service`

##########
File path: core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
##########
@@ -85,6 +86,24 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
     assert(diskBlockManager.getAllBlocks().isEmpty)
   }
 
+  test("should still create merge directories if one already exists under a local dir") {
+    val mergeDir0 = new File(rootDir0, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (!mergeDir0.exists()) {
+      Files.createDirectories(mergeDir0.toPath)
+    }
+    val mergeDir1 = new File(rootDir1, DiskBlockManager.MERGE_MANAGER_DIR)
+    if (mergeDir1.exists()) {
+      Utils.deleteRecursively(mergeDir1)
+    }
+    testConf.set("spark.local.dir", rootDirs)
+    testConf.set("spark.shuffle.push.enabled", "true")
+    testConf.set("spark.shuffle.service.enabled", "true")
+    diskBlockManager = new DiskBlockManager(testConf, deleteFilesOnStop = true)
+    Utils.getConfiguredLocalDirs(testConf).map(
+      rootDir => new File(rootDir, DiskBlockManager.MERGE_MANAGER_DIR))
+      .filter(mergeDir => mergeDir.exists())

Review comment:
       Add assertions?

##########
File path: core/src/test/scala/org/apache/spark/shuffle/HostLocalShuffleReadingSuite.scala
##########
@@ -133,4 +133,13 @@ class HostLocalShuffleReadingSuite extends SparkFunSuite with Matchers with Loca
       assert(remoteBytesRead.sum === 0 && remoteBlocksFetched.sum === 0)
     }
   }
+
+  test("Enable host local shuffle reading when Push based shuffle is enabled") {
+    val conf = new SparkConf()
+      .set(SHUFFLE_SERVICE_ENABLED, true)
+      .set("spark.yarn.maxAttempts", "1")
+      .set(PUSH_BASED_SHUFFLE_ENABLED, true)
+    sc = new SparkContext("local-cluster[2,1,1024]", "test-host-local-shuffle-reading", conf)

Review comment:
       nit: `local-cluster[2, 1, 1024]` (extra space after the comma)

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       IIUC, I think `dirs.get.isEmpty` is what you really need here.

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +370,51 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  override def getMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Option[Array[String]]): Seq[ManagedBuffer] = {
+    val indexFile =
+      getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }
+    // Number of chunks is number of indexes - 1
+    val numChunks = size / 8 - 1
+    for (index <- 0 until numChunks) yield {
+      new FileSegmentManagedBuffer(transportConf, dataFile,
+        offsets.get(index),
+        offsets.get(index + 1) - offsets.get(index))
+    }
+  }
+
+  /**
+   * This is only used for reading local merged block meta data.
+   */
+  override def getMergedBlockMeta(
+      blockId: ShuffleBlockId,
+      dirs: Option[Array[String]]): MergedBlockMeta = {
+    val indexFile =
+      getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val size = indexFile.length.toInt
+    val numChunks = (size / 8) - 1
+    val metaFile = getMergedBlockMetaFile(conf.getAppId, blockId.shuffleId, blockId.reduceId, dirs)
+    val chunkBitMaps = new FileSegmentManagedBuffer(transportConf, metaFile, 0L, metaFile.length)
+    new MergedBlockMeta(numChunks, chunkBitMaps)
+  }
+
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */

Review comment:
       Does this comment belong to `getMergedBlockData`?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       Why not use `dirs: Option[Array[String]]` directly?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-p", "-m" + permission, dirToCreate.getAbsolutePath)
+        val proc = builder.start()
+        val exitCode = proc.waitFor()
+        if (dirToCreate.exists()) {
+          created = dirToCreate
+        }
+        logDebug(
+          s"Created directory at ${dirToCreate.getAbsolutePath} and exitCode $exitCode")
+      } catch {
+        case e: SecurityException => created = null;

Review comment:
       Add a warning log with the exception?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       I think this comment is not appropriate for a util function. Shall we move this to the caller side in `DiskBlockManager`?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +729,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def getLocalMergedBlockMeta(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): MergedBlockMeta = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockMeta(blockId, Some(dirs))

Review comment:
       How do you plan test `getLocalMergedBlockData` and `getLocalMergedBlockMeta`?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,45 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          // This executor does not find merge_manager directory, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.
+          if (!mergeDir.exists()) {
+            logDebug(
+              s"Try to create $mergeDir and its sub dirs since the merge dir does not exist")

Review comment:
       nit: `merge` -> `merge_manager`?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/ExecutorShuffleInfo.java
##########
@@ -35,7 +35,11 @@
   public final String[] localDirs;
   /** Number of subdirectories created within each localDir. */
   public final int subDirsPerLocalDir;
-  /** Shuffle manager (SortShuffleManager) that the executor is using. */
+  /** Shuffle manager (SortShuffleManager) that the executor is using.

Review comment:
       ```suggestion
     /** 
      * Shuffle manager (SortShuffleManager) that the executor is using.
   ```

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +

Review comment:
       nit: `Failed to create directory ${dirToCreate.getAbsolutePath} with $permission...`?

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)

Review comment:
       nit: 2 indents

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -431,7 +432,7 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
       executorInfo.subDirsPerLocalDir));
   }
   private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
-    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+    return String.format("shuffleMerged_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,

Review comment:
       Shall we make this prefix a constant field?

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -1438,16 +1441,34 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
     assert(message.contains(expected))
   }
 
-  test("isPushBasedShuffleEnabled when both PUSH_BASED_SHUFFLE_ENABLED" +
-    " and SHUFFLE_SERVICE_ENABLED are true") {
+  test("isPushBasedShuffleEnabled when PUSH_BASED_SHUFFLE_ENABLED " +
+    "and SHUFFLE_SERVICE_ENABLED are both set to true in YARN mode with maxAttempts set to 1") {
     val conf = new SparkConf()
     assert(Utils.isPushBasedShuffleEnabled(conf) === false)
     conf.set(PUSH_BASED_SHUFFLE_ENABLED, true)
     conf.set(IS_TESTING, false)
     assert(Utils.isPushBasedShuffleEnabled(conf) === false)
     conf.set(SHUFFLE_SERVICE_ENABLED, true)
+    conf.set(SparkLauncher.SPARK_MASTER, "yarn")
+    conf.set("spark.yarn.maxAttempts", "1")
     assert(Utils.isPushBasedShuffleEnabled(conf) === true)
+    conf.set("spark.yarn.maxAttempts", "2")
+    assert(Utils.isPushBasedShuffleEnabled(conf) === false)
+  }
+
+  test("Test create dir with 770") {
+    val testDir = new File("target/testDir");
+    FileUtils.deleteQuietly(testDir)
+    Utils.createDirWithCustomizedPermission(testDir, "770")
+    val permission = PosixFilePermissions.toString(
+      JavaFiles.getPosixFilePermissions(Paths.get("target/testDir")))
+    assert(permission.equals("rwxrwx---"))
+    val foo = new File(testDir, "foo.txt")
+    Files.touch(foo)
+    assert(testDir.exists && testDir.isDirectory)
+    FileUtils.deleteQuietly(testDir)
   }
+

Review comment:
       Revert this?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.

Review comment:
       Have you addressed this comment?
   
   Looking at the code, it still looks inconsistent with the comment. I assume the comment should look like "Will create the merge_manager directory if it doesn't exist under the local dir." to match the code.
   
   

##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       I don't remember I recommended ever...but yes it's recommended to use wildcard imports when [there're more than  6 entities](https://github.com/databricks/scala-style-guide#imports).
   
   

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       This is not resolved? @zhouyejoe 

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,41 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.

Review comment:
       I'm fine with it.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r642027998



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(

Review comment:
       Can we keep this name consistent, that is, `getMergedBlockData` with what is in the `ShuffleBlockResolver` 

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -149,6 +178,12 @@ object BlockId {
       ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt)
     case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) =>
       ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED(appId, shuffleId, reduceId) =>

Review comment:
       Nit: Shall we make this consistent and call it `SHUFFLE_MERGED_DATA`?

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +370,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  override def getMergedBlockData(

Review comment:
       Nit: Missing javadoc
   ```
     /**
      * This is only used for reading local merged block data. In such cases, all chunks in the
      * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
      * knows how to consume local merged shuffle file as multiple chunks.
      */
      ```

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       When pushBasedShuffle is enabled, we want the hostLocalDirManager to be initialized. push-based shuffle works with old fetch protocol as well. Should this be:
   ```
   (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
             !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) )||
             Utils.isPushBasedShuffleEnabled(conf)
    ```         
    If yes, please add a UT as well to check if `hostLocalDirManager` is initialized when just push-based shuffle is enabled. I think we have this UT internally.

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +198,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {

Review comment:
       In our latest version it doesn't return anything.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -149,6 +178,12 @@ object BlockId {
       ShuffleIndexBlockId(shuffleId.toInt, mapId.toLong, reduceId.toInt)
     case SHUFFLE_PUSH(shuffleId, mapIndex, reduceId) =>
       ShufflePushBlockId(shuffleId.toInt, mapIndex.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED(appId, shuffleId, reduceId) =>
+      ShuffleMergedBlockId(appId, shuffleId.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED_INDEX(appId, shuffleId, reduceId) =>
+      ShuffleMergedIndexBlockId(appId, shuffleId.toInt, reduceId.toInt)
+    case SHUFFLE_MERGED_META(appId, shuffleId, reduceId) =>
+      ShuffleMergedMetaBlockId(appId, shuffleId.toInt, reduceId.toInt)

Review comment:
       Same here. Are we using this anywhere?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -129,6 +155,9 @@ object BlockId {
   val SHUFFLE_DATA = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).data".r
   val SHUFFLE_INDEX = "shuffle_([0-9]+)_([0-9]+)_([0-9]+).index".r
   val SHUFFLE_PUSH = "shufflePush_([0-9]+)_([0-9]+)_([0-9]+)".r
+  val SHUFFLE_MERGED = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).data".r
+  val SHUFFLE_MERGED_INDEX = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).index".r
+  val SHUFFLE_MERGED_META = "shuffleMerged_([_A-Za-z0-9]*)_([0-9]+)_([0-9]+).meta".r

Review comment:
       Where are we using this?

##########
File path: core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala
##########
@@ -85,6 +86,39 @@ class DiskBlockManagerSuite extends SparkFunSuite with BeforeAndAfterEach with B
     assert(diskBlockManager.getAllBlocks().isEmpty)
   }
 
+  test("find active merged shuffle directories") {

Review comment:
       This is a stale UT. It's not needed

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-p", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       `permission` variable is not being used. 
   Also we should still keep the TODO here 
   ```
     * TODO: Find out why can't we create a dir using java api with permission 770
      *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
      *  PosixFilePermissions.fromString("rwxrwx---"))
    
   ```
   Also I think we have generalized this createDir method but for more restrictive permissions, the java api would work.
   

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +198,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return Array.empty[File]
+        }
+      }
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (!mergeDir.exists()) {
+          logDebug(s"Creating $mergeDir as it does not exist")
+          // This executor didn't see merge_manager in the local dir, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.
+          try {
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              // Only one container will create this directory. The filesystem will handle any race
+              // conditions.
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))

Review comment:
       Should we not check the existence of the subdir before trying to create it?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -728,6 +736,27 @@ private[spark] class BlockManager(
     }
   }
 
+  /**
+   * Get the local merged shuffle block data for the given block ID as multiple chunks.
+   * A merged shuffle file is divided into multiple chunks according to the index file.
+   * Instead of reading the entire file as a single block, we split it into smaller chunks
+   * which will be memory efficient when performing certain operations.
+   */
+  def getLocalMergedBlockData(
+      blockId: ShuffleBlockId,
+      dirs: Array[String]): Seq[ManagedBuffer] = {
+    shuffleManager.shuffleBlockResolver.getMergedBlockData(blockId, Some(dirs))
+  }
+
+  /**
+   * Get the local merged shuffle block meta data for the given block ID.
+   */
+  def geLocalMergedBlockMeta(

Review comment:
       Same here, `getMergedBlockMeta`?

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +198,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          return Array.empty[File]
+        }
+      }
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (!mergeDir.exists()) {
+          logDebug(s"Creating $mergeDir as it does not exist")

Review comment:
       logDebug(s"Creating $mergeDir and its subdirs since the merge dir does not exist")

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -52,6 +59,17 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
   // of subDirs(i) is protected by the lock of subDirs(i)
   private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir))
 
+  // Get merge directory name, append attemptId if there is any
+  private val mergeDirName =
+    conf.getOption("spark.app.attempt.id")
+      .map(id => MERGE_DIRECTORY + "_" + id).getOrElse(MERGE_DIRECTORY)
+
+  /**
+   * Create merge directories
+   */
+  private[spark] val activeMergedShuffleDirs: Array[File] =
+    createLocalDirsForMergedShuffleBlocks(conf)

Review comment:
       Can you please check the signature of this method again? We don't need to keep the dirs in `DiskBlockManager` instance memory. We don't even use the `activeMergedShuffleDirs` anywhere in this class. This still looks like stale code.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r621391389



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       +CC @tgravescs for suggestions.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637970858



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (!mergeDir.exists()) {
+          logDebug(s"Creating $mergeDir as it does not exist")
+          // This executor didn't see merge_manager in the local dir, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.
+          try {
+            val mergeDir = new File(rootDir, mergeDirName)

Review comment:
       why not reuse the above `mergeDir`?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648875364



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       Added TODO in the comment that we have to use mkdir here.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648743781



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java
##########
@@ -45,18 +45,21 @@
 
   private final TransportClient client;
   private final String appId;
+  private final int attemptId;
   private final String[] blockIds;
   private final BlockFetchingListener listener;
   private final Map<String, ManagedBuffer> buffers;

Review comment:
       Will resolve the multi-attempts in SPARK-35546

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {
     // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
-      appPathsInfo.subDirsPerLocalDir, filename);
+    File targetFile = ExecutorDiskUtils.getFile(appAttemptPathsInfo.activeLocalDirs,
+      appAttemptPathsInfo.subDirsPerLocalDir, filename);
     logger.debug("Get merged file {}", targetFile.getAbsolutePath());
     return targetFile;
   }
 
-  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
-    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, fileName);
+  private File getMergedShuffleDataFile(String appId, int shuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, fileName);
   }
 
-  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
-    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, indexName);
+  private File getMergedShuffleIndexFile(String appId, int shuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, indexName);
   }
 
-  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
-    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
-    return getFile(appShuffleId.appId, metaName);
+  private File getMergedShuffleMetaFile(String appId, int shuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appId, shuffleId, reduceId));
+    return getFile(appId, metaName);
   }
 
   @Override
   public String[] getMergedBlockDirs(String appId) {
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.get(appId),
       "application " + appId + " is not registered or NM was restarted.");
-    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+    String[] activeLocalDirs = Preconditions.checkNotNull(appAttemptPathsInfo.activeLocalDirs,
       "application " + appId
       + " active local dirs list has not been updated by any executor registration");
     return activeLocalDirs;
   }
 
-  @Override
-  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
-    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
-    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
-    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
-      "application " + appId + " is not registered or NM was restarted.");
-    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+  /**
+   * Clean up the AppShufflePartitionInfo for a specific application attempt.
+   * If attemptId is -1, it means to clean up all the AppShufflePartitionInfo from
+   * all the attempts. Otherwise, only the AppShufflePartitionInfo from the specific
+   * application attempt will be cleaned up.
+   */
+  private void cleanupShufflePartitionInfo(String appId, int attemptId) {
+    Iterator<Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>>> iterator =
       partitions.entrySet().iterator();
     while (iterator.hasNext()) {
-      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
-      AppShuffleId appShuffleId = entry.getKey();
-      if (appId.equals(appShuffleId.appId)) {
+      Map.Entry<AppAttemptShuffleId, Map<Integer, AppAttemptShufflePartitionInfo>> entry = iterator.next();
+      AppAttemptShuffleId appAttemptShuffleId = entry.getKey();
+      if (appId.equals(appAttemptShuffleId.appId)
+          && (attemptId == -1 || attemptId == appAttemptShuffleId.attemptId)) {
         iterator.remove();
-        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+        for (AppAttemptShufflePartitionInfo partitionInfo : entry.getValue().values()) {
           partitionInfo.closeAllFiles();
         }
       }
     }
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppAttemptPathsInfo appAttemptPathsInfo = Preconditions.checkNotNull(appsPathsInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    cleanupShufflePartitionInfo(appId, -1);

Review comment:
       Will resolve the multi-attempts in SPARK-35546




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638028239



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -529,10 +530,17 @@ private[spark] class BlockManager(
 
   private def registerWithExternalShuffleServer(): Unit = {
     logInfo("Registering executor with local external shuffle service.")
+    val shuffleManagerMeta =
+      if (conf.get(config.PUSH_BASED_SHUFFLE_ENABLED)) {
+        s"${shuffleManager.getClass.getName}:" +
+          s"${diskBlockManager.getMergeDirectoryAndAttemptIDJsonString()}}}"

Review comment:
       After second thinking, it might be better to keep the consistent behavior under push-based shuffle for all the type of cluster managers.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644962656



##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       On second thought, given `Utils` is not available, there might not be an alternative location for this.
   We might need to look at a common infra module for utilities like this which all modules can depend on ... thoughts @Ngone51 ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640955976



##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -644,15 +644,15 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
 
     val tempDir2 = Utils.createTempDir()
     val sourceFile1 = new File(tempDir2, "foo.txt")
-    Files.touch(sourceFile1)
+    com.google.common.io.Files.touch(sourceFile1)

Review comment:
       Since this is scala, you can rename imported class.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644956403



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {

Review comment:
       Resolving this given we are moving multiple attempt support to a subsequent jira

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       On second thought, given `Utils` is not available, there might not be an alternative location for this.
   We might need to look at a common infra module for utilities like this which all modules can depend on ... thoughts @Ngone51 ?

##########
File path: core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala
##########
@@ -343,6 +359,50 @@ private[spark] class IndexShuffleBlockResolver(
     }
   }
 
+  /**
+   * This is only used for reading local merged block data. In such cases, all chunks in the
+   * merged shuffle file need to be identified at once, so the ShuffleBlockFetcherIterator
+   * knows how to consume local merged shuffle file as multiple chunks.
+   */
+  override def getMergedBlockData(blockId: ShuffleBlockId): Seq[ManagedBuffer] = {
+    val indexFile = getMergedBlockIndexFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    val dataFile = getMergedBlockDataFile(conf.getAppId, blockId.shuffleId, blockId.reduceId)
+    // Load all the indexes in order to identify all chunks in the specified merged shuffle file.
+    val size = indexFile.length.toInt
+    val buffer = ByteBuffer.allocate(size)
+    val offsets = buffer.asLongBuffer
+    val dis = new DataInputStream(Files.newInputStream(indexFile.toPath))
+    try {
+      dis.readFully(buffer.array)
+    } finally {
+      dis.close()
+    }

Review comment:
       On second thought, given `Utils` is not available, there might not be an alternative location for this.
   We might need to look at a common infra module for utilities like this which all modules can depend on (or is there something already ?) ... thoughts @Ngone51 ?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       I agree with @otterc's comment - though I think the suggested change in the comment does not do what you want it to do Chandni ? :-)

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (dirs.map(_.nonEmpty).getOrElse(false)) {
   ```

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (dirs.exists(_.nonEmpty)) {
   ```

##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -83,6 +88,33 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
 
   def getFile(blockId: BlockId): File = getFile(blockId.name)
 
+  /**
+   * This should be in sync with
+   * @see [[org.apache.spark.network.shuffle.RemoteBlockPushResolver#getFile(
+   *     java.lang.String, java.lang.String)]]
+   */
+  def getMergedShuffleFile(blockId: BlockId, dirs: Option[Array[String]]): File = {
+    blockId match {
+      case mergedBlockId: ShuffleMergedBlockId =>
+        getMergedShuffleFile(mergedBlockId.name, dirs)
+      case mergedIndexBlockId: ShuffleMergedIndexBlockId =>
+        getMergedShuffleFile(mergedIndexBlockId.name, dirs)
+      case mergedMetaBlockId: ShuffleMergedMetaBlockId =>
+        getMergedShuffleFile(mergedMetaBlockId.name, dirs)
+      case _ =>
+        throw new IllegalArgumentException(
+          s"Only merged block ID is supported, but got $blockId")
+    }
+  }
+
+  private def getMergedShuffleFile(filename: String, dirs: Option[Array[String]]): File = {
+    if (dirs.isEmpty) {

Review comment:
       ```suggestion
       if (!dirs.exists(_.nonEmpty)) {
   ```

##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -2566,11 +2601,28 @@ private[spark] object Utils extends Logging {
   }
 
   /**
-   * Push based shuffle can only be enabled when external shuffle service is enabled.
+   * Push based shuffle can only be enabled when the application is submitted
+   * to run in YARN mode, with external shuffle service enabled and
+   * spark.yarn.maxAttempts or the yarn cluster default max attempts is set to 1.
+   * TODO: SPARK-35546 Support push based shuffle with multiple app attempts
    */
   def isPushBasedShuffleEnabled(conf: SparkConf): Boolean = {
     conf.get(PUSH_BASED_SHUFFLE_ENABLED) &&
-      (conf.get(IS_TESTING).getOrElse(false) || conf.get(SHUFFLE_SERVICE_ENABLED))
+      (conf.get(IS_TESTING).getOrElse(false) ||
+        (conf.get(SHUFFLE_SERVICE_ENABLED) &&
+          conf.get(SparkLauncher.SPARK_MASTER, null) == "yarn") &&
+          getYarnMaxAttempts(conf) == 1)
+  }
+
+  /** Returns the maximum number of attempts to register the AM in YARN mode. */
+  def getYarnMaxAttempts(conf: SparkConf): Int = {
+      val sparkMaxAttempts = conf.getOption("spark.yarn.maxAttempts").map(_.toInt)
+      val yarnMaxAttempts = getSparkOrYarnConfig(conf, YarnConfiguration.RM_AM_MAX_ATTEMPTS,
+        YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS.toString).toInt
+      sparkMaxAttempts match {
+        case Some(x) => if (x <= yarnMaxAttempts) x else yarnMaxAttempts
+        case None => yarnMaxAttempts
+      }

Review comment:
       This already exists in spark right ? If yes, remove duplication ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648743696



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.

Review comment:
       Updated the comments accordingly.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r633295505



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Updated accordingly. Please help reivew. @mridulm @Ngone51 @otterc .
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637462835



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -414,23 +411,53 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         Longs.toArray(sizes));
     }
     partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == 0) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not 0, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message
+          // won't override the merge dirs. But it can be overridden by ExecutorRegister
+          // message from new app attempts.
+          appsPathInfo.compute(appId, (id, appAttemptPathsInfo) -> {
+            if (appAttemptPathsInfo != null && mergeDirectoryMeta.attemptId > appAttemptPathsInfo.attemptId) {
+              appAttemptPathsInfo = new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId,
+                executorInfo.localDirs, mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir);
+            }
+            return appAttemptPathsInfo;

Review comment:
       Added the remove part




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r633258762



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }
+      if (!mergeDirCreated) {
+        // This executor didn't see any merge_manager directories, it will start creating them.
+        // It's possible that the other executors launched at the same time may also reach here but
+        // we are working on the assumption that the executors launched around the same time will
+        // have the same set of application local directories.
+        localDirs.foreach { rootDir =>
+          try {
+            val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+            // Only one container will create this directory. The filesystem will handle any race
+            // conditions.
+            if (!mergeDir.exists()) {
+              Utils.createDirWith770(mergeDir)

Review comment:
       @otterc Added -p into the createDirWith770 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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615513625



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       @otterc If the merge_dir can get created under the blockmgr_UUID.randomUUID dir? Since RegisterExecutor message would send this blockmgr_UUID.randomUUID dir to ESS, ESS would know what are the local dirs to be used for merge dir. In our internal version, ESS will use the first RegisterExecutor message as the merge dirs list to be used. Suppose the scenario described: Executor1 gets the local dirs list "/grid/[a-c]/yarn/usercache/test/appcache/application_id/", Executor2 gets the local dirs list "/grid/[d-f]". Either of them will create the merge_dir under their own local dirs, for example: Executor1 creates "/grid/[a-c]/........./merge_dir"  and Executor2 creates "/grid/[d-f]//........./merge_dir". Executor1 is lucky to be the the first one to register with the local ESS, ESS will only use the dirs for "/grid/[a-c]". But during the Executor registration, the actual dirs in the message is "/grid/a/yarn/tmp/usercache/testuser/appCache/application_id/blockmgr_RandomID", "/grid/b/
 yarn/tmp/usercache/testuser/appCache/application_id/blockmgr_RandomID",
   "/grid/c/yarn/tmp/usercache/testuser/appCache/application_id/blockmgr_RandomID". Internally, we trim the dirs to the "/grid/[a_c]/yarn/tmp/usercache/testuser/appCache/application_id/", assume the merge_dirs are "/grid/[a_c]/yarn/tmp/usercache/testuser/appCache/application_id/mergedirs". Even though there are other "/grid/[d-f]//........./merge_dir" created by Executor2, but since ESS will only use the dirs from the Executor1, those dirs will not be used. With the same logic, if we move merge_dir to subdir of blockmgr_RandomIDs, this will still work, right? The only cons would be there would be empty merge_dirs/subdirs created by other executors.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640347580



##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -644,15 +644,15 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
 
     val tempDir2 = Utils.createTempDir()
     val sourceFile1 = new File(tempDir2, "foo.txt")
-    Files.touch(sourceFile1)
+    com.google.common.io.Files.touch(sourceFile1)

Review comment:
       Have to add this to distinguish between com.google.common.io.Files and java.nio.file.Files, for the added unit test to check the permission on the created dir. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640348273



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.

Review comment:
       Added UT for both valid and invalid JSON, with/without multiple attempts. Please check.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627590658



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       @zhouyejoe If we are sending the merge directory name via RegisterExecutor message which is the 5th step, then I don't think 4th step is necessary where the executor deletes directory of old attempt.
   Currently, the executors don't delete the previous previous attempts block manager dirs and neither does the shuffle service. It is deleted by Yarn when the application is finally finished. All the blockMgr dirs hang around until the app is done.
   With merge_manager directory it will be the same case. In this solution, the remote shuffle service doesn't depend on  listing the dirs so what do we get from having the executor delete old attempt merge_manager dir?
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-810691560


   Can one of the admins verify this patch?


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] asfgit closed pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #32007:
URL: https://github.com/apache/spark/pull/32007


   


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r615345038



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This is a valid point. I think we can add UUID.randomUUID to the merge folder path, same as the original shuffle local files dirs.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r627646535



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       To add to my comment above (should have provided more context): I was assuming we are relying on directory existence to infer latest.
   But I like @Ngone51's idea here better:
   >  And if the executor becomes the one who creates the merge dir, we send the ExecutorShuffleInfo with the special shuffleManager, e.g., "sort_merge_manager_attemptX". And ExternalBlockHandler can parse the shuffleManager into two parts
   
   `shuffleManager` should to be relevant when we had multiple shuffle managers - and from evolution point of view, we could always introduce a new shuffle in future.
   My proposal is extension of the idea above - here, attempt is metadata about shuffleManager we want to convey.
   Ideally, this should be within `ExecutorShuffleInfo` - but given the compatibility issues here, why not add a general way to encode metadata about the shuffle manager here ?
   
   That is, instead of hardcoding 'SortShuffleManager_attemptX' and 'SortShuffleManager' as supported patterns, we could simply allow for 'SortShuffleManager' (existing) and 'SortShuffleManager:json_string' ?
   This can allow for any potential future evolution to also be possible - with currently `{"merge_dir": "merge_directory_<attemptid>", "attempt_id":<attempt_id>}` being the initial field (@zhouyejoe let us explicitly pass the directory name, instead of inferring it from attempt id ?)
   
   Ofcourse if ':' is missing in shuffleManager, then we treat it as empty metadata case.
   
   With this in place, we would still need changes to push block protocol to include attempt id - but given no one is using that currently, we can make that change.
   With this explicit specification of merge directory, we remove all ambiguity.
   
   Thoughts @Ngone51 ?
   
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640309963



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -19,8 +19,6 @@
 
 import com.google.common.base.Objects;
 import io.netty.buffer.ByteBuf;
-import org.apache.commons.lang3.builder.ToStringBuilder;

Review comment:
       These two imports are not being used so I removed it. I think we need to clean up the unused import here since this is not original code but we introduced for push based shuffle. 




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Victsm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r640956322



##########
File path: core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
##########
@@ -17,12 +17,13 @@
 
 package org.apache.spark.util
 
-import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
-  FileOutputStream, PrintStream, SequenceInputStream}
+import java.io._

Review comment:
       Nit: should keep the original import list and just add the new class you are importing.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r637973198



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +204,60 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Array[File] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      for (rootDir <- configuredLocalDirs) {
+        val mergeDir = new File(rootDir, mergeDirName)
+        if (!mergeDir.exists()) {
+          logDebug(s"Creating $mergeDir as it does not exist")
+          // This executor didn't see merge_manager in the local dir, it will start creating them.
+          // It's possible that the other executors launched at the same time may also reach here
+          // but we are working on the assumption that the executors launched around the same time
+          // will have the same set of application local directories.
+          try {
+            val mergeDir = new File(rootDir, mergeDirName)
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              // Only one container will create this directory. The filesystem will handle any race
+              // conditions.
+              val sudDir = new File(mergeDir, "%02x".format(dirNum))
+              Utils.createDirWithCustomizedPermission(sudDir, "770")
+            }
+            logInfo(s"Merge directory at $mergeDir")
+          } catch {
+            case e: IOException =>
+              logError(
+                s"Failed to create merge dir in $rootDir. Ignoring this directory.", e)
+          }
+        }
+      }
+    }
+    findActiveMergedShuffleDirs(conf)
+  }
+
+  private def findActiveMergedShuffleDirs(conf: SparkConf): Array[File] = {

Review comment:
       This is only called once. Shall we inline it into `createLocalDirsForMergedShuffleBlocks`?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r647789581



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       This will be handled as part of SPARK-30602, resolving thread.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644956403



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -211,63 +214,74 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI
 
   /**
    * The logic here is consistent with
-   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   * @see [[org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile(org.apache.spark.storage.BlockId)]]
    */
   private File getFile(String appId, String filename) {

Review comment:
       Resolving this given we are moving multiple attempt support to a subsequent jira




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638033883



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -415,24 +429,65 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc
         bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
         Longs.toArray(sizes));
     }
-    partitions.remove(appShuffleId);
-    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    partitions.remove(appAttemptShuffleId);
+    logger.info("Finalized shuffle {} from Application {}_{}.", msg.shuffleId, msg.appId, msg.attemptId);
     return mergeStatuses;
   }
 
   @Override
   public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
     if (logger.isDebugEnabled()) {
       logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
-        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
-          executorInfo.subDirsPerLocalDir);
+        + "num sub-dirs {} shuffleManager {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir, executorInfo.shuffleManager);
+    }
+    String shuffleManagerMeta = executorInfo.shuffleManager;
+    if (shuffleManagerMeta.contains(":")) {
+      String mergeDirInfo = shuffleManagerMeta.substring(shuffleManagerMeta.indexOf(":") + 1);
+      try {
+        ObjectMapper mapper = new ObjectMapper();
+        MergeDirectoryMeta mergeDirectoryMeta = mapper.readValue(mergeDirInfo, MergeDirectoryMeta.class);
+        if (mergeDirectoryMeta.attemptId == -1) {
+          // When attemptId is 0, there is no attemptId stored in the ExecutorShuffleInfo.
+          // Only the first ExecutorRegister message can register the merge dirs
+          appsPathsInfo.computeIfAbsent(appId, id ->
+            new AppAttemptPathsInfo(appId, mergeDirectoryMeta.attemptId, executorInfo.localDirs,
+              mergeDirectoryMeta.mergeDir, executorInfo.subDirsPerLocalDir));
+        } else {
+          // If attemptId is not -1, there is attemptId stored in the ExecutorShuffleInfo.
+          // The first ExecutorRegister message from the same application attempt will
+          // register the merge dirs in Shuffle Service. Any later ExecutorRegister message

Review comment:
       nit: "Shuffle Service" -> "external shuffle service"




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r626212566



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       > ESS cannot create the merge_manager directory under application local directory because it doesn't have permissions to do so.
   
   Good point, thanks.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r648748667



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +185,82 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the external shuffle service doesn't have
+   * permission to create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(): Unit = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under the local dir.
+      Utils.getConfiguredLocalDirs(conf).foreach { rootDir =>
+        try {
+          val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+          if (!mergeDir.exists()) {
+            // This executor does not find merge_manager directory, it will try to create
+            // the merge_manager directory and the sub directories.
+            logDebug(s"Try to create $mergeDir and its sub dirs since the " +
+              s"$MERGE_MANAGER_DIR dir does not exist")
+            for (dirNum <- 0 until subDirsPerLocalDir) {
+              val subDir = new File(mergeDir, "%02x".format(dirNum))
+              if (!subDir.exists()) {
+                // Only one container will create this directory. The filesystem will handle
+                // any race conditions.
+                createDirWithCustomizedPermission(subDir, "770")
+              }
+            }
+          }
+          logInfo(s"Merge directory and its sub dirs get created at $mergeDir")
+        } catch {
+          case e: IOException =>
+            logError(
+              s"Failed to create $MERGE_MANAGER_DIR dir in $rootDir. Ignoring this directory.", e)
+        }
+      }
+    }
+  }
+
+  /**
+   * Create a directory that is writable by the group.
+   * Grant the customized permission so the shuffle server can
+   * create subdirs/files within the merge folder.
+   * TODO: Find out why can't we create a dir using java api with permission 770
+   *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
+   *  PosixFilePermissions.fromString("rwxrwx---")))
+   */
+  def createDirWithCustomizedPermission(dirToCreate: File, permission: String): Unit = {

Review comment:
       @Ngone51 Thought on this? I think this is a valid point. If not due to umask in Yarn, we don't have to use "mkdir" to create the dir, but with the PosixFilePermissions and Files.createDirectory.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r644967308



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -504,7 +504,8 @@ private[spark] class BlockManager(
 
     hostLocalDirManager = {
       if (conf.get(config.SHUFFLE_HOST_LOCAL_DISK_READING_ENABLED) &&
-          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) {
+          !conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL) ||
+          Utils.isPushBasedShuffleEnabled(conf)) {

Review comment:
       I agree with @otterc's comment - though I think the suggested change in the comment does not do what you want it to do Chandni ? :-)




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r614521383



##########
File path: core/src/main/scala/org/apache/spark/util/Utils.scala
##########
@@ -315,6 +315,38 @@ private[spark] object Utils extends Logging {
     dir.getCanonicalFile
   }
 
+  /**
+   * Create a directory that is writable by the group.
+   * Grant 770 permission so the shuffle server can create subdirs/files within the merge folder.
+   */
+  def createDirWith770(dirToCreate: File): Unit = {
+    var attempts = 0
+    val maxAttempts = MAX_DIR_CREATION_ATTEMPTS
+    var created: File = null
+    while (created == null) {
+      attempts += 1
+      if (attempts > maxAttempts) {
+        throw new IOException(
+          s"Failed to create directory ${dirToCreate.getAbsolutePath} after " +
+            s"${maxAttempts} attempts!")
+      }
+      try {
+        val builder = new ProcessBuilder().command(
+          "mkdir", "-m770", dirToCreate.getAbsolutePath)

Review comment:
       @mridulm I've tried this last year and it doesn't work. It still creates the directory with permission `750`.  Internally we have this  TODO as well for this method.
   ```
      * TODO: Find out why can't we create a dir using java api with permission 770
      *  Files.createDirectories(mergeDir.toPath, PosixFilePermissions.asFileAttribute(
      *  PosixFilePermissions.fromString("rwxrwx---")))
      ```
   This has something to do with yarn setting a umask when it starts the container process. I don't remember the details because I tried this early last year.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on pull request #32007:
URL: https://github.com/apache/spark/pull/32007#issuecomment-853810536


   I left a few comments, most are minor. It's a good move to handle issues separately.


-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] Ngone51 commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r628200689



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Using JSON as metadata is really a good idea! I'm +1 with it.




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] zhouyejoe commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
zhouyejoe commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r633258762



##########
File path: core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala
##########
@@ -153,6 +189,59 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea
     }
   }
 
+  /**
+   * Get the list of configured local dirs storing merged shuffle blocks created by executors
+   * if push based shuffle is enabled. Note that the files in this directory will be created
+   * by the external shuffle services. We only create the merge_manager directories and
+   * subdirectories here because currently the shuffle service doesn't have permission to
+   * create directories under application local directories.
+   */
+  private def createLocalDirsForMergedShuffleBlocks(conf: SparkConf): Option[Array[File]] = {
+    if (Utils.isPushBasedShuffleEnabled(conf)) {
+      // Will create the merge_manager directory only if it doesn't exist under any local dir.
+      val localDirs = Utils.getConfiguredLocalDirs(conf)
+      var mergeDirCreated = false;
+      for (rootDir <- localDirs) {
+        val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+        if (mergeDir.exists()) {
+          logDebug(s"Not creating $mergeDir as it already exists")
+          mergeDirCreated = true
+        }
+      }
+      if (!mergeDirCreated) {
+        // This executor didn't see any merge_manager directories, it will start creating them.
+        // It's possible that the other executors launched at the same time may also reach here but
+        // we are working on the assumption that the executors launched around the same time will
+        // have the same set of application local directories.
+        localDirs.foreach { rootDir =>
+          try {
+            val mergeDir = new File(rootDir, MERGE_MANAGER_DIR)
+            // Only one container will create this directory. The filesystem will handle any race
+            // conditions.
+            if (!mergeDir.exists()) {
+              Utils.createDirWith770(mergeDir)

Review comment:
       @otterc Added -p into the createDirWith770 method.

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -87,6 +87,29 @@ case class ShufflePushBlockId(shuffleId: Int, mapIndex: Int, reduceId: Int) exte
   override def name: String = "shufflePush_" + shuffleId + "_" + mapIndex + "_" + reduceId
 }
 
+@DeveloperApi
+case class ShuffleMergedBlockId(appId: String, shuffleId: Int, reduceId: Int) extends BlockId {
+  override def name: String = "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".data"
+}
+
+@DeveloperApi
+case class ShuffleMergedIndexBlockId(
+  appId: String,
+  shuffleId: Int,
+  reduceId: Int) extends BlockId {
+  override def name: String =
+    "mergedShuffle_" + appId + "_" + shuffleId + "_" + reduceId + ".index"

Review comment:
       Updated accordingly. Please help reivew. @mridulm @Ngone51 @otterc .
   




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #32007: [SPARK-33350][SHUFFLE] Add support to DiskBlockManager to create merge directory and to get the local shuffle merged data

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #32007:
URL: https://github.com/apache/spark/pull/32007#discussion_r638399675



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -419,4 +419,8 @@ public long mergedIndexCacheSize() {
   public int ioExceptionsThresholdDuringMerge() {
     return conf.getInt("spark.shuffle.server.ioExceptionsThresholdDuringMerge", 4);
   }
+
+  public int appAttemptId() {
+    return conf.getInt("spark.app.attempt.id", -1);

Review comment:
       @Ngone51 Curious if you mean about the assumption of parsing attempt as an `int` ? Or simply about this (application attempts) being used only in yarn currently ?




-- 
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org