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



##########
File path: 
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -128,49 +151,100 @@ protected AppShuffleInfo 
validateAndGetAppShuffleInfo(String appId) {
   }
 
   /**
-   * Given the appShuffleInfo, shuffleId and reduceId that uniquely identifies 
a given shuffle
-   * partition of an application, retrieves the associated metadata. If not 
present and the
-   * corresponding merged shuffle does not exist, initializes the metadata.
+   * Given the appShuffleInfo, shuffleId, shuffleMergeId and reduceId that 
uniquely identifies
+   * a given shuffle partition of an application, retrieves the associated 
metadata. If not
+   * present and the corresponding merged shuffle does not exist, initializes 
the metadata.
    */
   private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(
       AppShuffleInfo appShuffleInfo,
       int shuffleId,
-      int reduceId) {
-    File dataFile = appShuffleInfo.getMergedShuffleDataFile(shuffleId, 
reduceId);
-    ConcurrentMap<Integer, Map<Integer, AppShufflePartitionInfo>> partitions =
+      int shuffleMergeId,
+      int reduceId) throws RuntimeException {
+    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()) {
-            return null;
-          }
-          return new ConcurrentHashMap<>();
+    AtomicReference<Map<Integer, Map<Integer, AppShufflePartitionInfo>>> 
shuffleMergePartitionsRef
+      = new AtomicReference<>(null);
+    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);
+        shuffleMergePartitionsRef.set(newShuffleMergePartitions);
+        return newShuffleMergePartitions;
+      } else if (shuffleMergePartitionsMap.containsKey(shuffleMergeId)) {
+        shuffleMergePartitionsRef.set(shuffleMergePartitionsMap);
+        return shuffleMergePartitionsMap;
+      } else {
+        int latestShuffleMergeId = shuffleMergePartitionsMap.keySet().stream()
+          .mapToInt(x -> x).max().orElse(UNDEFINED_SHUFFLE_MERGE_ID);
+        int secondLatestShuffleMergeId = 
shuffleMergePartitionsMap.keySet().stream()
+          .mapToInt(x -> x).filter(x -> x != latestShuffleMergeId)
+          .max().orElse(UNDEFINED_SHUFFLE_MERGE_ID);
+        if (latestShuffleMergeId > shuffleMergeId) {
+          // Reject the request as we have already seen a higher 
shuffleMergeId than the
+          // current incoming one
+          throw new RuntimeException(String.format("Rejecting shuffle blocks 
push request for"
+            + " shuffle %s with shuffleMergeId %s for application %s_%s as a 
higher"
+              + " shuffleMergeId %s request is already seen", shuffleId, 
shuffleMergeId,
+              appShuffleInfo.appId, appShuffleInfo.attemptId, 
latestShuffleMergeId));
         } else {
-          return map;
+          // 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 (latestShuffleMergeId != UNDEFINED_SHUFFLE_MERGE_ID &&
+              shuffleMergePartitionsMap.containsKey(latestShuffleMergeId)) {
+            Map<Integer, AppShufflePartitionInfo> latestShufflePartitions =
+              shuffleMergePartitionsMap.get(latestShuffleMergeId);
+            mergedShuffleCleaner.execute(() ->
+              closeAndDeletePartitionFiles(latestShufflePartitions));
+            shuffleMergePartitionsMap.put(latestShuffleMergeId, 
STALE_SHUFFLE_PARTITIONS);
+          }
+          // Remove older shuffleMergeIds which won't be required anymore
+          if (secondLatestShuffleMergeId != UNDEFINED_SHUFFLE_MERGE_ID) {
+            shuffleMergePartitionsMap.remove(secondLatestShuffleMergeId);
+          }
+          Map<Integer, AppShufflePartitionInfo> newShufflePartitions = new 
ConcurrentHashMap<>();
+          shuffleMergePartitionsMap.put(shuffleMergeId, newShufflePartitions);
+          shuffleMergePartitionsRef.set(shuffleMergePartitionsMap);
+          return shuffleMergePartitionsMap;
         }
-      });
-    if (shufflePartitions == null) {
-      return null;
+      }
+    });
+
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+        shuffleMergePartitionsRef.get().get(shuffleMergeId);
+    if (null == shufflePartitions || isStaleOrTooLate(shufflePartitions)) {
+      // It only gets here when shufflePartitions is either 
FINALIZED_SHUFFLE_PARTITIONS or
+      // STALE_SHUFFLE_PARTITIONS or null. 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 shufflePartitions == FINALIZED_SHUFFLE_PARTITIONS ?
+        FINALIZED_APP_SHUFFLE_PARTITION_INFO : 
STALE_APP_SHUFFLE_PARTITION_INFO;

Review comment:
       Lets say if we have `shuffle Id` = 0 and latest `shuffleMergeId` = 3 and 
if there is a push request for `shuffleMergeId` = 2 then the key 
`shuffleMergeId = 2` in the Map would be pointing to 
`STALE_SHUFFLE_PARTITIONS`, therefore this would return 
`STALE_APP_SHUFFLE_PARTITION_INFO` 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.

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