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 2022/11/21 07:03:43 UTC

[GitHub] [spark] mridulm commented on a diff in pull request #37922: [SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished

mridulm commented on code in PR #37922:
URL: https://github.com/apache/spark/pull/37922#discussion_r1027637860


##########
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java:
##########
@@ -393,6 +394,35 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
     }
   }
 
+  @Override
+  public void removeShuffleMerge(RemoveShuffleMerge msg) {
+    String appId = msg.appId;
+    int appAttemptId = msg.appAttemptId;
+    int shuffleId = msg.shuffleId;
+    AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
+    if (appShuffleInfo.attemptId != appAttemptId) {
+      throw new IllegalArgumentException(
+          String.format("The attempt id %s in this RemoveShuffleMerge message does not match "
+                  + "with the current attempt id %s stored in shuffle service for application %s",
+              appAttemptId, appShuffleInfo.attemptId, appId));
+    }
+
+    appShuffleInfo.shuffles.compute(shuffleId, (shuffleIdKey, partitionsInfo) -> {
+      if (null != partitionsInfo) {

Review Comment:
   The validation related to `shuffleMergeId`, from `finalizeShuffleMerge`, is applicable here depending on `shuffleMergeId`.
   I have given the details [here](https://github.com/apache/spark/pull/37922#discussion_r990753031) - please refer to it.



##########
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/NoOpMergedShuffleFileManager.java:
##########
@@ -84,4 +85,9 @@ public MergedBlockMeta getMergedBlockMeta(
   public String[] getMergedBlockDirs(String appId) {
     throw new UnsupportedOperationException("Cannot handle shuffle block merge");
   }
+
+  @Override
+  public void removeShuffleMerge(RemoveShuffleMerge removeShuffleMerge) {
+    throw new UnsupportedOperationException("Cannot handle shuffle block merge");

Review Comment:
   nit: 
   ```suggestion
       throw new UnsupportedOperationException("Cannot handle merged shuffle remove");
   ```



##########
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java:
##########
@@ -121,6 +122,13 @@ MergedBlockMeta getMergedBlockMeta(
    */
   String[] getMergedBlockDirs(String appId);
 
+  /**
+   * Remove shuffle merge data files.
+   *
+   * @param removeShuffleMerge Remove shuffle merge RPC

Review Comment:
   ```suggestion
      * @param removeShuffleMerge contains shuffle details (appId, shuffleId, etc) to uniquely identify a shuffle to be removed
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

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