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



##########
File path: 
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -135,51 +150,87 @@ protected AppShuffleInfo 
validateAndGetAppShuffleInfo(String appId) {
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleInfo appShuffleInfo,
       int shuffleId,
+      int shuffleMergeId,
       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>> shuffleMergePartitions 
=
+      partitions.compute(shuffleId, (id, shuffleMergePartitionsMap) -> {
+        if (shuffleMergePartitionsMap == null) {
+          logger.info("Creating a new attempt for shuffle blocks push request 
for"
+              + " shuffle {} with shuffleMergeId {} for application {}_{}", 
shuffleId,
+              shuffleMergeId, appShuffleInfo.appId, appShuffleInfo.attemptId));
+          Map<Integer, Map<Integer, AppShufflePartitionInfo>> 
newShuffleMergePartitions
+            = new ConcurrentHashMap<>();
+          Map<Integer, AppShufflePartitionInfo> newPartitionsMap = new 
ConcurrentHashMap<>();
+          newShuffleMergePartitions.put(shuffleMergeId, newPartitionsMap);
+          return newShuffleMergePartitions;
+        } else if (shuffleMergePartitionsMap.containsKey(shuffleMergeId)) {
+          return shuffleMergePartitionsMap;
+        } else {
+          int latestShuffleMergeId = 
shuffleMergePartitionsMap.keySet().stream()
+            .mapToInt(v -> v).max().orElse(UNDEFINED_SHUFFLE_MERGE_ID);
+          if (latestShuffleMergeId > shuffleMergeId) {
+            logger.info("Rejecting shuffle blocks push request for shuffle {} 
with"
+                + " shuffleMergeId {} for application {}_{} as a higher 
shuffleMergeId"
+                + " {} request is already seen", shuffleId, shuffleMergeId,
+                appShuffleInfo.appId, appShuffleInfo.attemptId, 
latestShuffleMergeId));
+            // Reject the request as we have already seen a higher 
shuffleMergeId than the
+            // current incoming one
             return null;
+          } else {
+            // Higher shuffleMergeId seen for the shuffle ID meaning new stage 
attempt is being
+            // run for the shuffle ID. Close and clean up old shuffleMergeId 
files,
+            // happens in the non-deterministic stage retries
+            logger.info("Creating a new attempt for shuffle blocks push 
request for"
+               + " shuffle {} with shuffleMergeId {} for application {}_{} 
since it is"
+               + " higher than the latest shuffleMergeId {} already seen", 
shuffleId,
+               shuffleMergeId, appShuffleInfo.appId, appShuffleInfo.attemptId,
+               latestShuffleMergeId));
+            if (null != shuffleMergePartitionsMap.get(latestShuffleMergeId)) {
+              Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+                shuffleMergePartitionsMap.get(latestShuffleMergeId);
+              mergedShuffleCleaner.execute(() ->
+                closeAndDeletePartitionFiles(shufflePartitions));
+            }
+            shuffleMergePartitionsMap.put(latestShuffleMergeId, 
STALE_SHUFFLE_PARTITIONS);
+            Map<Integer, AppShufflePartitionInfo> newPartitionsMap = new 
ConcurrentHashMap<>();
+            shuffleMergePartitionsMap.put(shuffleMergeId, newPartitionsMap);
+            return shuffleMergePartitionsMap;
           }
-          return new ConcurrentHashMap<>();
-        } else {
-          return map;
         }
       });
-    if (shufflePartitions == null) {
+
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = 
shuffleMergePartitions.get(shuffleMergeId);
+    if (shufflePartitions == FINALIZED_SHUFFLE_PARTITIONS
+        || shufflePartitions == STALE_SHUFFLE_PARTITIONS) {
+      // It only gets here when shufflePartitions is either 
FINALIZED_SHUFFLE_PARTITIONS or STALE_SHUFFLE_PARTITIONS.
+      // This happens in 2 cases:
+      // 1. Incoming block request is for an older shuffleMergeId of a shuffle 
(i.e already higher shuffle
+      // sequence Id blocks are being merged for this shuffle Id.
+      // 2. Shuffle for the current shuffleMergeId is already finalized.
       return null;
     }
 
+    File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, 
shuffleMergeId, 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, shuffleMergeId, 
reduceId);
       File metaFile =
-        appShuffleInfo.getMergedShuffleMetaFile(shuffleId, reduceId);
+        appShuffleInfo.getMergedShuffleMetaFile(shuffleId, shuffleMergeId, 
reduceId);
       try {
-        if (dataFile.exists()) {
-          return null;
-        } else {
-          return newAppShufflePartitionInfo(
-            appShuffleInfo.appId, shuffleId, reduceId, dataFile, indexFile, 
metaFile);
-        }
+        return newAppShufflePartitionInfo(appShuffleInfo.appId, shuffleId, 
shuffleMergeId,
+          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);
+            + "shuffleMergeId %s reduceId %s", appShuffleInfo.appId, 
shuffleId, shuffleMergeId, reduceId), e);
       }
     });

Review comment:
       Since we are not removing the `shuffleMergeId` key from the map even for 
the cases of determinate stages. It should be fine, but once we remove it then 
probably we might need the file existence check. Anyways I brought back the 
data file existence 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.

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