spark-reviews mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] [spark] Ngone51 commented on a change in pull request #33034: [SPARK-32923][CORE][SHUFFLE] Handle indeterminate stage retries for push-based shuffle
Date Thu, 22 Jul 2021 15:05:24 GMT

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



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockId.scala
##########
@@ -172,11 +195,15 @@ object BlockId {
   val SHUFFLE_BATCH = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)_([0-9]+)".r
   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_DATA = "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
-  val SHUFFLE_CHUNK = "shuffleChunk_([0-9]+)_([0-9]+)_([0-9]+)".r
+  val SHUFFLE_PUSH = "shufflePush_([0-9]+)_(-?[0-9]+)_([0-9]+)_([0-9]+)".r

Review comment:
       `shuffleSequenceId` can be negative?

##########
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 =

Review comment:
       Could you add a comment to explain the semantics of this complex map?

##########
File path: core/src/main/scala/org/apache/spark/shuffle/ShuffleBlockPusher.scala
##########
@@ -90,11 +93,12 @@ private[spark] class ShuffleBlockPusher(conf: SparkConf) extends Logging
{
       dataFile: File,
       partitionLengths: Array[Long],
       dep: ShuffleDependency[_, _, _],
-      mapIndex: Int): Unit = {
+      mapIndex: Int,
+      shuffleSequenceId: Int): Unit = {

Review comment:
       Get this from `dep` instead of using an explicit param?

##########
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) {

Review comment:
       Print a log for each case?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -410,12 +503,17 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg)
throws IOExc
           + "with the current attempt id %s stored in shuffle service for application %s",
           msg.appAttemptId, appShuffleInfo.attemptId, msg.appId));
     }
-    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
-      appShuffleInfo.partitions.remove(msg.shuffleId);
+    Map<Integer, Map<Integer, AppShufflePartitionInfo>> shuffleSequencePartitions
=
+        appShuffleInfo.partitions.get(msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = shuffleSequencePartitions.get(msg.shuffleSequenceId);
+    if (shufflePartitions != INVALID_SHUFFLE_PARTITIONS) {
+      shuffleSequencePartitions.put(msg.shuffleSequenceId, FINALIZED_SHUFFLE_PARTITIONS);
+      appShuffleInfo.partitions.put(msg.shuffleId, shuffleSequencePartitions);
+    }
     MergeStatuses mergeStatuses;
     if (shufflePartitions == null || shufflePartitions.isEmpty()) {
       mergeStatuses =
-        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+        new MergeStatuses(msg.shuffleId, msg.shuffleSequenceId, new RoaringBitmap[0], new
int[0], new long[0]);

Review comment:
       We'd return the `MergeStatuses` even if it's a finalize request from an older shuffle?
   
   

##########
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:
       nit: 2 indents.
   
   Please also check other places.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -379,7 +467,12 @@ public void onData(String streamId, ByteBuffer buf) {
 
         @Override
         public void onComplete(String streamId) {
-          if (isTooLate) {
+          if (isInvalidBlock) {
+            // 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.INVALID_BLOCK_PUSH));

Review comment:
       How about `isStaleBlock`, `STALE_BLOCK_PUS`, etc, instead of using the word "invalid"?

   
   I personally think that "invalid" is not accurate here.

##########
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) {

Review comment:
       nit: line len > 100.
   
   Please also check other places.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/MergeStatuses.java
##########
@@ -40,6 +40,11 @@
 public class MergeStatuses extends BlockTransferMessage {
   /** Shuffle ID **/
   public final int shuffleId;
+  /**
+   * shuffleSequenceId is to track the sequence of shuffle executions
+   * This is required to handle indeterminate stage retries
+   **/

Review comment:
       nit: `**/` -> `*/`

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -353,18 +438,21 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream
msg) {
     // getting killed. When this happens, we need to distinguish the duplicate blocks as
they
     // arrive. More details on this is explained in later comments.
 
+    // Track if the block is received from an older shuffleSequenceId attempt.
+    final boolean isInvalidBlock = partitionInfoBeforeCheck == INVALID_SHUFFLE_PARTITIONS;

Review comment:
       As mentioned in the above comment, `partitionInfoBeforeCheck` is `null` for this case?
And you also compare with two different types here...
   
   (same for FINALIZED_SHUFFLE_PARTITIONS) 

##########
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);

Review comment:
       IIUC, it's equivalent to use `shuffleSequencePartitionsMap.keys.max` here?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlockChunks.java
##########
@@ -37,14 +37,18 @@
   public final int[] reduceIds;
   // The i-th int[] in chunkIds contains all the chunks for the i-th reduceId in reduceIds.
   public final int[][] chunkIds;
+  // shuffleSequenceId is to uniquely identify a stage attempt of a shuffle ID.

Review comment:
       nit: "indeterminate stage attempt"
   
   Please also check 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.

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


Mime
View raw message