venkata91 commented on a change in pull request #33034:
URL: https://github.com/apache/spark/pull/33034#discussion_r675759111



##########
File path: 
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -135,102 +151,144 @@ protected AppShuffleInfo 
validateAndGetAppShuffleInfo(String appId) {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleInfo appShuffleInfo,
       int shuffleId,
+      int shuffleSequenceId,
       int reduceId) {
-    File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, 
reduceId);
-    ConcurrentMap<Integer, Map<Integer, AppShufflePartitionInfo>> partitions =
+    ConcurrentMap<Integer, Map<Integer, Map<Integer, 
AppShufflePartitionInfo>>> partitions =
       appShuffleInfo.partitions;
-    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
-      partitions.compute(shuffleId, (id, map) -> {
-        if (map == null) {
-          // If this partition is already finalized then the partitions map 
will not contain the
-          // shuffleId but the data file would exist. In that case the block 
is considered late.
-          if (dataFile.exists()) {
+    Map<Integer, Map<Integer, AppShufflePartitionInfo>> 
shuffleSequencePartitions =
+      partitions.compute(shuffleId, (id, shuffleSequencePartitionsMap) -> {
+        if (shuffleSequencePartitionsMap == null) {
+          Map<Integer, Map<Integer, AppShufflePartitionInfo>> 
newShuffleSequencePartitions = new ConcurrentHashMap<>();
+          Map<Integer, AppShufflePartitionInfo> newPartitionsMap = new 
ConcurrentHashMap<>();
+          newShuffleSequencePartitions.put(shuffleSequenceId, 
newPartitionsMap);
+          return newShuffleSequencePartitions;
+        } else if 
(shuffleSequencePartitionsMap.containsKey(shuffleSequenceId)) {
+          return shuffleSequencePartitionsMap;
+        } else {
+          int latestShuffleSequenceID = 
computeLatestShuffleSequenceId(appShuffleInfo, shuffleId);
+          if (latestShuffleSequenceID > shuffleSequenceId) {
+            // Reject the request as we have already seen a higher 
shuffleSequenceId than the current
+            // incoming one
             return null;
+          } else {
+            // Higher shuffleSequenceId seen for the shuffle ID meaning new 
stage attempt is being
+            // run for the shuffle ID. Close and clean up old 
shuffleSequenceId files,
+            // happens in the non-deterministic stage retries
+            if (null != 
shuffleSequencePartitionsMap.get(latestShuffleSequenceID)) {
+              Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+                  shuffleSequencePartitionsMap.get(latestShuffleSequenceID);
+              mergedShuffleCleaner.execute(() ->
+                  closeAndDeletePartitionFiles(shufflePartitions));
+            }
+            shuffleSequencePartitionsMap.put(latestShuffleSequenceID, 
INVALID_SHUFFLE_PARTITIONS);
+            Map<Integer, AppShufflePartitionInfo> newPartitionsMap = new 
ConcurrentHashMap<>();
+            shuffleSequencePartitionsMap.put(shuffleSequenceId, 
newPartitionsMap);
+            return shuffleSequencePartitionsMap;
           }
-          return new ConcurrentHashMap<>();
-        } else {
-          return map;
         }
       });
-    if (shufflePartitions == null) {
+
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = 
shuffleSequencePartitions.get(shuffleSequenceId);
+    if (shufflePartitions == FINALIZED_SHUFFLE_PARTITIONS || shufflePartitions 
== INVALID_SHUFFLE_PARTITIONS) {
+      // It only gets here when shufflePartitions is either 
FINALIZED_SHUFFLE_PARTITIONS or INVALID_SHUFFLE_PARTITIONS.
+      // This happens in 2 cases:
+      // 1. Incoming block request is for an older shuffleSequenceId of a 
shuffle (i.e already higher shuffle
+      // sequence Id blocks are being merged for this shuffle Id.
+      // 2. Shuffle for the current shuffleSequenceId is already finalized.
       return null;
     }
 
+    File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, 
shuffleSequenceId, reduceId);
     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.
+      // It only gets here when the key is not present in the map. The first 
time the merge
+      // manager receives a pushed block for a given application shuffle 
partition.
       File indexFile =
-        appShuffleInfo.getMergedShuffleIndexFile(shuffleId, reduceId);
+        appShuffleInfo.getMergedShuffleIndexFile(shuffleId, shuffleSequenceId, 
reduceId);
       File metaFile =
-        appShuffleInfo.getMergedShuffleMetaFile(shuffleId, reduceId);
+        appShuffleInfo.getMergedShuffleMetaFile(shuffleId, shuffleSequenceId, 
reduceId);
       try {
-        if (dataFile.exists()) {
-          return null;
-        } else {
-          return newAppShufflePartitionInfo(
-            appShuffleInfo.appId, shuffleId, reduceId, dataFile, indexFile, 
metaFile);
-        }
+        return newAppShufflePartitionInfo(appShuffleInfo.appId, shuffleId, 
shuffleSequenceId, 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", appShuffleInfo.appId, shuffleId, reduceId), e);
+            + "shuffleSequenceId %s reduceId %s", appShuffleInfo.appId, 
shuffleId, shuffleSequenceId, reduceId), e);
       }
     });
   }
 
+  private int computeLatestShuffleSequenceId(AppShuffleInfo appShuffleInfo, 
int shuffleId) {
+    if (appShuffleInfo.partitions.get(shuffleId) != null) {
+      Set<Integer> shuffleSequenceIds = 
appShuffleInfo.partitions.get(shuffleId).keySet();
+      return shuffleSequenceIds.stream().mapToInt(v -> 
v).max().orElse(UNDEFINED_SHUFFLE_SEQUENCE_ID);
+    } else {
+      return UNDEFINED_SHUFFLE_SEQUENCE_ID;
+    }
+  }
+
   @VisibleForTesting
   AppShufflePartitionInfo newAppShufflePartitionInfo(
       String appId,
       int shuffleId,
+      int shuffleSequenceId,
       int reduceId,
       File dataFile,
       File indexFile,
       File metaFile) throws IOException {
-    return new AppShufflePartitionInfo(appId, shuffleId, reduceId, dataFile,
+    return new AppShufflePartitionInfo(appId, shuffleId, shuffleSequenceId, 
reduceId, dataFile,
       new MergeShuffleFile(indexFile), new MergeShuffleFile(metaFile));
   }
 
   @Override
-  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int 
reduceId) {
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int 
shuffleSequenceId, int reduceId) {
     AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
+    if (appShuffleInfo.partitions.get(shuffleId).get(shuffleSequenceId) == 
INVALID_SHUFFLE_PARTITIONS) {
+      throw new RuntimeException(String.format(
+         "MergedBlock meta fetch for shuffleId %s shuffleSequenceId %s 
reduceId %s is %s", shuffleId,
+          shuffleSequenceId, reduceId, 
ErrorHandler.BlockFetchErrorHandler.INVALID_BLOCK_FETCH));
+    }
     File indexFile =
-      appShuffleInfo.getMergedShuffleIndexFile(shuffleId, reduceId);
+      appShuffleInfo.getMergedShuffleIndexFile(shuffleId, shuffleSequenceId, 
reduceId);
     if (!indexFile.exists()) {
       throw new RuntimeException(String.format(
         "Merged shuffle index file %s not found", indexFile.getPath()));
     }
     int size = (int) indexFile.length();
     // First entry is the zero offset
     int numChunks = (size / Long.BYTES) - 1;
-    File metaFile = appShuffleInfo.getMergedShuffleMetaFile(shuffleId, 
reduceId);
+    File metaFile = appShuffleInfo.getMergedShuffleMetaFile(shuffleId, 
shuffleSequenceId, reduceId);
     if (!metaFile.exists()) {
       throw new RuntimeException(String.format("Merged shuffle meta file %s 
not found",
         metaFile.getPath()));
     }
     FileSegmentManagedBuffer chunkBitMaps =
       new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
     logger.trace(
-      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, 
numChunks);
+      "{} shuffleId {} shuffleSequenceId {} reduceId {} num chunks {}",
+        appId, shuffleId, shuffleSequenceId, reduceId, numChunks);
     return new MergedBlockMeta(numChunks, chunkBitMaps);
   }
 
   @SuppressWarnings("UnstableApiUsage")
   @Override
-  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int 
reduceId, int chunkId) {
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int 
shuffleSequenceId, int reduceId, int chunkId) {
     AppShuffleInfo appShuffleInfo = validateAndGetAppShuffleInfo(appId);
-    File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, 
reduceId);
+    if (appShuffleInfo.partitions.get(shuffleId).get(shuffleSequenceId) == 
INVALID_SHUFFLE_PARTITIONS) {
+      throw new RuntimeException(String.format(
+          "MergedBlock data fetch for shuffleId %s shuffleSequenceId %s 
reduceId %s is %s", shuffleId,

Review comment:
       Thanks. Continuation indent is set to 4 spaces which I think is 
consistent with this one - 
https://google.github.io/styleguide/javaguide.html#s4.5.2-line-wrapping-indent




-- 
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-unsubscr...@spark.apache.org

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



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

Reply via email to