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 2020/10/15 20:21:19 UTC

[GitHub] [spark] otterc opened a new pull request #30062: [Spark 32916][Shuffle] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

otterc opened a new pull request #30062:
URL: https://github.com/apache/spark/pull/30062


   ### What changes were proposed in this pull request?
   This is one of the patches for SPIP [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602) which is needed for push-based shuffle.
   Summary of changes:
   - Adds an implementation of `MergedShuffleFileManager` which was introduced with [Spark 32915](https://issues.apache.org/jira/browse/SPARK-32915).
   - Integrated the push-based shuffle service with `YarnShuffleService`.
   
   ### Why are the changes needed?
   Refer to the SPIP in  [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   ### How was this patch tested?
   Added unit tests.
   The reference PR with the consolidated changes covering the complete implementation is also provided in [SPARK-30602](https://issues.apache.org/jira/browse/SPARK-30602).
   We have already verified the functionality and the improved performance as documented in the SPIP doc.
   
   Lead-authored-by: Min Shen mshen@linkedin.com
   Co-authored-by: Chandni Singh chsingh@linkedin.com
   Co-authored-by: Ye Zhou yezhou@linkedin.com
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721916361






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721566228


   **[Test build #130595 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130595/testReport)** for PR 30062 at commit [`7cf38c4`](https://github.com/apache/spark/commit/7cf38c4ad1fd3f62c46b0ff3f9b48490b281085c).


----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517475508



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";

Review comment:
       This is a copy-paste of the line: https://github.com/apache/spark/blob/7cf38c4ad1fd3f62c46b0ff3f9b48490b281085c/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java#L42
   
   I suggest to remove this one and use the `OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX`




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-720251110






----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516776712



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);

Review comment:
       The`msg.reduceId` is also not needed as it is the same as `partitionInfo.reduceId` (or the other way around).




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-716254033






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712512443


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34632/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709607544






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514003770



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partition.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            try {
+              partition.closeAllFiles();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partition.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public final FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    private final DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512851976



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       @attilapiros right now using map index is necessary to keep track of which shuffle block has been merged for a given shuffle partition tracked inside the bitmap.
   If it's a deterministic stage, then we do not want to distinguish 2 attempts of the same map task as 2 separate blocks, which could lead to data duplication.
   If it's a nondeterministic stage, then we will need to rerun the entire shuffle map stage and redo the merge.
   This will be handled by SPARK-32923.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718409141






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719155227


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130423/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721389401


   **[Test build #130576 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130576/testReport)** for PR 30062 at commit [`27e0d31`](https://github.com/apache/spark/commit/27e0d31a0e2a7350edcf7c388a44ab73d0cdc5fa).


----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r508170165



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       > If we write the data into separate files, another overhead is that the time taken for finalization will increase. 
   
   Make sense to me. Then, how about we merging them at the time the `onComplete()` is called?
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r507985489



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the

Review comment:
       Are you referring to the configuration `maxRemoteBlockSizeFetchToMem`? 
   We are aware that when this configuration is set and if a request is larger than this, the block will be saved to disk. 
   With push-based shuffle, data of a remote merged block is always large. If we don't divide it into chunks, the remote merged data will always be written to disk and then read from it again. This adds a lot more time.
   Also any failure during fetching an entire merged block will be much more costly. With the approach of dividing a merged block into size-able chunks
   - We don't have to write to the disk always so the runtime of jobs are shorter.
   - When fetch of a shuffle chunk fails, then we fallback to the original blocks corresponding to the mapIds which are part of this chunk.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722055068






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721896453


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35211/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514003367



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;

Review comment:
       Done

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r518998753



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX,
+      appShuffleId.shuffleId, msg.mapIndex, msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",

Review comment:
       Want to add that one of the things we are considering is the necessity of a bounded in-memory buffer on the server side to help with writing the blocks to merged files.
   The consideration is more from the aspect of how much potential improvement we could see on the disk write I/O when merging the blocks, and less about reaching better merge ratio for blocks.
   These 2 things are however very related though, and will be something that we will consider holistically inside SPARK-33331.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516167838



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -23,24 +23,31 @@
 import org.apache.spark.network.protocol.Encoders;
 
 // Needed by ScalaDoc. See SPARK-7726
-import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
 
 
 /**
  * Request to push a block to a remote shuffle service to be merged in push based shuffle.
  * The remote shuffle service will also include this message when responding the push requests.
  */
 public class PushBlockStream extends BlockTransferMessage {
+  public static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
   public final String appId;
-  public final String blockId;
+  public final int shuffleId;
+  public final int mapIndex;
+  public final int reduceId;
   // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of
   // blocks to be pushed.
   public final int index;
+  public final String streamId;

Review comment:
       Where's this streamId used?
   ShuffleId is used later as streamId in toString and decode.
   That seems a bit confusing.

##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,38 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * Class name of the implementation of MergedShuffleFileManager that merges the blocks
+   * pushed to it when push-based shuffle is enabled. Default implementation for merging the blocks
+   * remotely is 'org.apache.spark.network.shuffle.RemoteBlockPushResolver'.
+   * To turn off push-based shuffle at a cluster level, set the configuration to
+   * 'org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager'.
+   */
+  public String mergeShuffleFileManagerImpl() {
+    return conf.get("spark.shuffle.push.based.mergedShuffleFileManagerImpl",
+      "org.apache.spark.network.shuffle.RemoteBlockPushResolver");

Review comment:
       Should the default be NoOpMergedShuffleFileManager instead?




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-720230639






----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511051284



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the
+   * clients. Instead of serving the entire merged file, the shuffle service serves the
+   * merged file in `chunks`. A `chunk` constitutes few shuffle blocks in entirety and this
+   * configuration controls how big a chunk can get. A corresponding index file for each merged
+   * shuffle file will be generated indicating chunk boundaries.
+   */
+  public int minChunkSizeInMergedShuffleFile() {
+    return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+      conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m")));

Review comment:
       One thing we discussed internally is whether this config should be a server side config or a client side config.
   As @otterc mentioned, there are multiple reasons we break a merged shuffle partition file into multiple smaller chunks.
   One of the biggest reasons is to parallelize fetching shuffle data and task execution.
   If we have a multi-GB merged shuffle partition and the client is fetching it as a single block, then the client would wait until it fetches its entirety before handing off to the task processing logic to process the block, which is not ideal.
   The question is that whether size of the chunk should be a global configuration on the server side, irrespective of individual applications, or a Spark app configuration so users can fine tune it.
   We currently make it a server side config so we don't introduce another parameter for users to tune.
   Want to get inputs from the community on this as well.
   cc @Ngone51 @attilapiros @jiangxb1987 @tgravescs 




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723562188


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722694164


   The tests failures are not related to this change.


----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512232873



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.

Review comment:
       what do you mean by synchronously?  do you mean serially? 




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712518597






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712536767






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709572516


   Can one of the admins verify this patch?


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723358170


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35352/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721968391


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130612/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512427399



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -94,6 +95,9 @@
   static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure";
   private static final boolean DEFAULT_STOP_ON_FAILURE = false;
 
+  // Used by shuffle merge manager to create merged shuffle files.
+  protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/";

Review comment:
       While making changes to support sub-dir, I realized that `RemoteBlockPushResolver` doesn't really need this 
   `usercache/%s/appcache/%s` format.
   In one of our older versions when we were **not** using the  `executorShuffleInfo` from the `registerExecutor` message, we added this. However, now it uses the `localDirs` from `executorShuffleInfo` which are paths of the `blockManagerDirs`. The current implementation finds the root local dir by `localDir.substring(0, "usercache/{userId}/appcache/{appId/")` which seems un-necessary. What we need is to find the parent directory of each `blockManagerDir` and construct the path of each `merge_manager` directory, which is a sibling of `blockManagerDir`. Even though this is not that clean but it is similar to what we have now and doesn't change the `registerExecutor` protocol.
   
   Although this may increase the memory footprint of `appPathsInfo` map slightly.
   In an `appPathInfo`, instead of saving the complete local paths, for example 
   `[localDir1/usercache/{userId}/appcache/{appId}/merge_manager, localDir2/usercache/{userId}/appcache/{appId}/merge_manager/]`, currently we only save
   `[localDir1, localDir2]`.  While constructing the target file path we add the sub-part `usercache/{userId}/appcache/{appId}`.
   The target file path would be constructing using these parts: `{localDir}/{usercache/%s/appcache/%s/merge_manager}/filename`. 
   I do think this increase will not be that significant. 
   It would be much better to remove this assumption on the path format.
   
   @Victsm @zhouyejoe  @tgravescs @Ngone51 @attilapiros @mridulm 




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510468306



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {

Review comment:
       It's not used for test only. This will be used when merged shuffle data is fetched which will be part of future PRs.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509922973



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -94,6 +95,9 @@
   static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure";
   private static final boolean DEFAULT_STOP_ON_FAILURE = false;
 
+  // Used by shuffle merge manager to create merged shuffle files.
+  protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/";

Review comment:
       This is only used inside `RemoteBlockPushResolver`. Could we declare it as a static global valuable there?




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517760472



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.

Review comment:
       Done




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513246088



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -172,7 +178,9 @@ protected void serviceInit(Configuration conf) throws Exception {
       }
 
       TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf));
-      blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile);
+      shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH);

Review comment:
       +1 to add config on the server-side, you can leave the default value to `RemoteBlockPushResolver`. Thanks @otterc 




----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512667696



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partition.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            try {
+              partition.closeAllFiles();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partition.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),

Review comment:
       You know ahead the size of the **bitmaps** (and all the others too: **reduceIds**, **sizes**), but in case of the **bitmaps** this will be even an unnecessary array copy too which can be easily avoided by allocating those arrays with the right size and fill them up meanwhile an index is incremented.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721916361






----------------------------------------------------------------
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.

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


[GitHub] [spark] dongjoon-hyun commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725733508


   Hi, Guys.
   This seems to break `master` branch with `Hadoop 2.7`.
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/1061/testReport/junit/org.apache.spark.network.yarn/YarnShuffleServiceSuite/testCreateDefaultMergedShuffleFileManagerInstance/history/
   
   ![Screen Shot 2020-11-11 at 4 05 29 PM](https://user-images.githubusercontent.com/9700541/98878396-bc8b9e80-2437-11eb-8dc4-82ebf1b62735.png)
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-716254033






----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725904598


   Can you make a followup patch for this @otterc ?
   Please do test it locally on both hadoop-2.7 and default, thanks !


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718406581


   **[Test build #130395 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130395/testReport)** for PR 30062 at commit [`b3fc7d0`](https://github.com/apache/spark/commit/b3fc7d09188fd0507d9e7484ee4836addd19c97a).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  static class StreamCallbackImpl implements StreamCallbackWithID `


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r507481819



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);

Review comment:
       The `buf` here could be a slice of a larger byte buffer. Since we are caching these `bufs` here, because they cannot be written at this point, we don't want to cache the entire larger byte buffer. Copying the subsets of bytes we need will save memory.
   @Victsm




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721968179






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-717730159


   **[Test build #130357 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130357/testReport)** for PR 30062 at commit [`67cfa83`](https://github.com/apache/spark/commit/67cfa83f31ec8c921f0d1a7291f3189299bf484e).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719122307


   **[Test build #130427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130427/testReport)** for PR 30062 at commit [`6037d5e`](https://github.com/apache/spark/commit/6037d5e68165edb02962c57e3a55e15914763d5c).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709809399


   **[Test build #129869 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129869/testReport)** for PR 30062 at commit [`dceae72`](https://github.com/apache/spark/commit/dceae72f4a1719972cda23c8ea9f2309c129c4dd).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723544814






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709714308


   Fixed the lint errors in `RemoteBlockPushResolverSuite`.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722502275


   > Why would the error happen? Does it mean we calculate the wrong message length somewhere when allocating the buffer?
   
   In `RemoteBlockPushResolver` we re-use the `trackerBuf` when we serialize the `chunkTracker`. The code is [here](https://github.com/linkedin/spark/blob/a8dd6f58fe65db34770ac4165192188fe3b98639/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java#L864).
   Once the bytes are written to the file,  the `trackerBuf` is cleared.
   `trackerBuf` is a java heap buffer with an initial capacity which can expand its capacity on demand.
   
   In the magnet-upstream branch it was using `buf.writeBytes(bytes)` to do this. It's [here](https://github.com/linkedin/spark/blob/7478a3edff46b77d325d30bd952d6ba0a2f479ff/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java#L102). The implementation of `AbstractByteBuf.writeBytes` also calls `ensureWritable` before writing to the buf.
   ```
       @Override
       public ByteBuf writeBytes(byte[] src, int srcIndex, int length) {
           ensureWritable(length);
           setBytes(writerIndex, src, srcIndex, length);
           writerIndex += length;
           return this;
       }
   ``` 


----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513664646



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;

Review comment:
       Took that previous comment back.
   For checking shuffle partition getting finalized in the middle, we used a different condition instead of the null check.
   I think this suggestion to create a separate simpler callback just for the null case is a good idea, to simplify this part of the logic a bit more.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719139405






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719404772


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35051/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-717744130


   **[Test build #130357 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130357/testReport)** for PR 30062 at commit [`67cfa83`](https://github.com/apache/spark/commit/67cfa83f31ec8c921f0d1a7291f3189299bf484e).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513615760



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       @attilapiros I'm not sure I fully understand your question.
   The `blockId` inside `PushBlockStream` is shuffle partition block Id.
   This is the same as the the usage of `blockId` for fetching blocks:
   https://github.com/apache/spark/blob/a744fea3be12f1a53ab553040b95da730210bc88/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockStoreClient.java#L86-L92
   
   As for using a typed message instead of a string to represent the block id, do you see the same issue in the existing block fetching API as well?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510626481



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       IIRC, the `block2` is also stored in the shuffle data file at the client-side? So, if we choose to re-send the `block2`, we will also read it again from the disk, right? So I think the trade-off here is read IO + network transfer (retry) vs write IO + read IO (write to the file). As I mentioned earlier, I really can not tell which way is obviously better without any experiment. 




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709581026


   **[Test build #129856 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129856/testReport)** for PR 30062 at commit [`f34935f`](https://github.com/apache/spark/commit/f34935f911563f2bca7e151f8429a1a339fa2137).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719122307


   **[Test build #130427 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130427/testReport)** for PR 30062 at commit [`6037d5e`](https://github.com/apache/spark/commit/6037d5e68165edb02962c57e3a55e15914763d5c).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512229942



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,899 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(String.format(
+        "Merged shuffle index file %s of %s not found", indexFile.getPath(), id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle meta file %s of %s not found",
+        metaFile.getPath(), id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(id);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s of %s not found",
+        dataFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s of %s", indexFile.getPath(), id.toString()),
+          e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart

Review comment:
       The client will fail to fetch the merged shuffle data. However, the client will fallback to the original shuffle blocks that were part of the merged shuffle. So, eventually the client will succeed to fetch shuffle data.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516969462



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.streamId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final PushBlockStream msg;
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        PushBlockStream msg,
+        AppShuffleId appShuffleId,
+        int reduceId,
+        int mapIndex,
+        AppShufflePartitionInfo partitionInfo) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.msg = Preconditions.checkNotNull(msg);
+      this.appShuffleId = appShuffleId;
+      this.reduceId = reduceId;
+      this.mapIndex = mapIndex;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+    }
+
+    @Override
+    public String getID() {
+      return msg.streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+            updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", msg.streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            msg.streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+      } else {
+        logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }
+      Encoders.Bitmaps.encode(trackerBuf, chunkTracker);
+      long metaStartPos = metaChannel.position();
+      try {
+        logger.trace("{} shuffleId {} reduceId {} mapIndex {} write chunk to meta file",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex);
+        metaChannel.write(trackerBuf.nioBuffer());
+      } catch (IOException ioe) {
+        logger.warn("{} shuffleId {} reduceId {} mapIndex {} reset position of meta file to {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex, metaStartPos);
+        metaChannel.position(metaStartPos);
+        throw ioe;
+      } finally {
+        trackerBuf.clear();
+      }
+    }
+
+    void closeAllFiles() {
+      try {
+        if (dataChannel != null) {
+          dataChannel.close();
+          dataChannel = null;
+        }
+      } catch (IOException ioe) {
+        logger.warn("Error closing data channel for {} shuffleId {} reduceId {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId);
+      }
+      try {
+        if (metaChannel != null) {
+          metaChannel.close();
+          metaChannel = null;
+        }
+      } catch (IOException ioe) {
+        logger.warn("Error closing meta channel for {} shuffleId {} reduceId {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId);
+      }
+      try {
+        if (indexWriteStream != null) {
+          indexWriteStream.close();
+          indexWriteStream = null;
+        }
+      } catch (IOException ioe) {
+        logger.warn("Error closing index stream for {} shuffleId {} reduceId {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId);
+      }
+      try {
+        if (indexChannel != null) {
+          indexChannel.close();
+          indexChannel = null;
+        }
+      } catch (IOException ioe) {
+        logger.warn("Error closing index channel for {} shuffleId {} reduceId {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId);
+      }
+    }
+
+    @Override
+    protected void finalize() throws Throwable {
+      closeAllFiles();
+    }
+  }
+
+  /**
+   * Wraps all the information related to the merge directory of an application.
+   */
+  private static class AppPathsInfo {
+
+    private String[] activeLocalDirs;
+    private int subDirsPerLocalDir;
+
+    private AppPathsInfo updateActiveLocalDirs(
+        String appId,
+        String[] localDirs,
+        int subDirsPerLocalDir) {
+      if (activeLocalDirs == null) {
+        activeLocalDirs = Arrays.stream(localDirs)
+          .map(localDir ->
+            // Merge directory is created at the same level as block-manager directory. The list of
+            // local directories that we get from executorShuffleInfo are paths of each
+            // block-manager directory. To find out the merge directory location, we first find the
+            // parent dir and then append the "merger_manager" directory to it.
+            Paths.get(localDir).getParent().resolve(MERGE_MANAGER_DIR).toFile().getPath())
+          .toArray(String[]::new);
+        this.subDirsPerLocalDir = subDirsPerLocalDir;
+        if (logger.isInfoEnabled()) {
+          logger.info("Updated the active local dirs {} for application {}",
+            Arrays.toString(activeLocalDirs), appId);
+          }

Review comment:
       Fixed




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721859821


   **[Test build #130610 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130610/testReport)** for PR 30062 at commit [`7f8303a`](https://github.com/apache/spark/commit/7f8303ab5a3fda52e4748646867cb82088135bdf).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722047828


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35229/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516969116



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,490 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.concurrent.Semaphore;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private Path[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = createLocalDirs(2);
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf);
+    registerExecutor(TEST_APP, prepareLocalDirs(localDirs));
+  }
+
+  @After
+  public void after() {
+    try {
+      for (Path local : localDirs) {
+        FileUtils.deleteDirectory(local.toFile());
+      }
+      removeApplication(TEST_APP);
+    } catch (Exception e) {
+      // don't fail if clean up doesn't succeed.
+      log.debug("Error while tearing down", e);
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    } catch (Throwable t) {
+      assertTrue(t.getMessage().startsWith("Merged shuffle index file"));
+      Throwables.propagate(t);
+    }
+  }
+
+  @Test
+  public void testBasicBlockMerge() throws IOException {
+    PushBlock[] pushBlocks = new PushBlock[] {new PushBlock(0, 0, 0), new PushBlock(0, 1, 0)};
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, blocks);
+    MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
+      new FinalizeShuffleMerge(TEST_APP, 0));
+    validateMergeStatuses(statuses, new int[] {0}, new long[] {9});
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDividingMergedBlocksIntoChunks() throws IOException {
+    PushBlock[] pushBlocks = new PushBlock[] {
+      new PushBlock(0, 0, 0),
+      new PushBlock(0, 1, 0),
+      new PushBlock(0, 2, 0),
+      new PushBlock(0, 3, 0)
+    };
+    ByteBuffer[] buffers = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[2]),
+      ByteBuffer.wrap(new byte[3]),
+      ByteBuffer.wrap(new byte[5]),
+      ByteBuffer.wrap(new byte[3])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, buffers);
+    MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
+      new FinalizeShuffleMerge(TEST_APP, 0));
+    validateMergeStatuses(statuses, new int[] {0}, new long[] {13});
+    MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}});
+  }
+
+  @Test
+  public void testFinalizeWithMultipleReducePartitions() throws IOException {
+    PushBlock[] pushBlocks = new PushBlock[] {
+      new PushBlock(0, 0, 0),
+      new PushBlock(0, 1, 0),
+      new PushBlock(0, 0, 1),
+      new PushBlock(0, 1, 1)};
+    ByteBuffer[] buffers = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[2]),
+      ByteBuffer.wrap(new byte[3]),
+      ByteBuffer.wrap(new byte[5]),
+      ByteBuffer.wrap(new byte[3])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, buffers);
+    MergeStatuses statuses = pushResolver.finalizeShuffleMerge(
+      new FinalizeShuffleMerge(TEST_APP, 0));
+    validateMergeStatuses(statuses, new int[] {0, 1}, new long[] {5, 8});
+    MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, meta, new int[]{5}, new int[][]{{0, 1}});
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+    // stream 2 has more data and then completes
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException {
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+    // stream 2 now completes completes
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOException {
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    // This should be ignored
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
+  }
+
+  @Test
+  public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOException {
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    // This should be ignored
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+    // stream 2 now completes completes
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
+  }
+
+  @Test
+  public void testFailureAfterData() throws IOException {
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    assertEquals("num-chunks", 0, blockMeta.getNumChunks());
+  }
+
+  @Test
+  public void testFailureAfterMultipleDataBlocks() throws IOException {
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    assertEquals("num-chunks", 0, blockMeta.getNumChunks());
+  }
+
+  @Test
+  public void testFailureAfterComplete() throws IOException {
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onComplete(stream.getID());
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
+  }
+
+  @Test (expected = RuntimeException.class)
+  public void testTooLateArrival() throws IOException {
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
+      new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    for (ByteBuffer block : blocks) {
+      stream.onData(stream.getID(), block);
+    }
+    stream.onComplete(stream.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
+      new PushBlockStream(TEST_APP, 0, 1, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4]));
+    try {
+      stream1.onComplete(stream1.getID());
+    } catch (RuntimeException re) {
+      assertEquals(
+        "Block shufflePush_0_1_0 received after merged shuffle is finalized",
+          re.getMessage());
+      MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+      validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
+      throw re;
+    }
+  }
+
+  @Test
+  public void testIncompleteStreamsAreOverwritten() throws IOException {
+    registerExecutor(TEST_APP, prepareLocalDirs(localDirs));
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4]));
+    // There is a failure
+    stream1.onFailure(stream1.getID(), new RuntimeException("forced error"));
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5]));
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5}, new int[][]{{1}});
+  }
+
+  @Test (expected = RuntimeException.class)
+  public void testFailureWith3Streams() throws IOException {
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 0, 0, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, 0, 1, 0, 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0));
+    // There is a failure with stream2
+    stream2.onFailure(stream2.getID(), new RuntimeException("forced error"));
+    StreamCallbackWithID stream3 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 2, 0, 0));
+    // This should be deferred
+    stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[5]));
+    // Since this stream didn't get any opportunity it will throw couldn't find opportunity error
+    RuntimeException failedEx = null;
+    try {
+      stream3.onComplete(stream2.getID());
+    } catch (RuntimeException re) {
+      assertEquals(
+        "Couldn't find an opportunity to write block shufflePush_0_2_0 to merged shuffle",
+        re.getMessage());
+      failedEx = re;
+    }
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4}, new int[][] {{0}});
+    if (failedEx != null) {
+      throw failedEx;
+    }
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUpdateLocalDirsOnlyOnce() throws IOException {
+    String testApp = "updateLocalDirsOnlyOnceTest";
+    Path[] activeLocalDirs = createLocalDirs(1);
+    registerExecutor(testApp, prepareLocalDirs(activeLocalDirs));
+    assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
+    assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
+      activeLocalDirs[0].toFile().getPath()));
+    // Any later executor register from the same application should not change the active local
+    // dirs list
+    Path[] updatedLocalDirs = localDirs;
+    registerExecutor(testApp, prepareLocalDirs(updatedLocalDirs));
+    assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
+    assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
+      activeLocalDirs[0].toFile().getPath()));
+    removeApplication(testApp);
+    try {
+      pushResolver.getMergedBlockDirs(testApp);
+    } catch (Throwable e) {
+      assertTrue(e.getMessage()
+        .startsWith("application " + testApp + " is not registered or NM was restarted."));
+      Throwables.propagate(e);
+    }
+  }
+
+  @Test
+  public void testCleanUpDirectory() throws IOException, InterruptedException {
+    String testApp = "cleanUpDirectory";
+    Semaphore deleted = new Semaphore(0);
+    pushResolver = new RemoteBlockPushResolver(conf) {
+      @Override
+      void deleteExecutorDirs(Path[] dirs) {
+        super.deleteExecutorDirs(dirs);
+        deleted.release();
+      }
+    };
+    Path[] activeDirs = createLocalDirs(1);
+    registerExecutor(testApp, prepareLocalDirs(activeDirs));
+    PushBlock[] pushBlockIds = new PushBlock[] {new PushBlock(0, 0, 0)};
+    ByteBuffer[] blocks = new ByteBuffer[] {ByteBuffer.wrap(new byte[4])};
+    pushBlockHelper(testApp, pushBlockIds, blocks);
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(testApp, 0, 0);
+    validateChunks(testApp, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
+    String[] mergeDirs = pushResolver.getMergedBlockDirs(testApp);
+    pushResolver.applicationRemoved(testApp,  true);
+    // Since the cleanup happen in a different thread, check few times to see if the merge dirs gets
+    // deleted.
+    deleted.acquire();
+    for (String mergeDir : mergeDirs) {
+      Assert.assertFalse(Files.exists(Paths.get(mergeDir)));
+    }
+  }
+
+  private Path[] createLocalDirs(int numLocalDirs) throws IOException {
+    Path[] localDirs = new Path[numLocalDirs];
+    for (int i = 0; i < localDirs.length; i++) {
+      localDirs[i] = Files.createTempDirectory("shuffleMerge");
+      localDirs[i].toFile().deleteOnExit();
+    }
+    return localDirs;
+  }
+
+  private void registerExecutor(String appId, String[] localDirs) throws IOException {
+    ExecutorShuffleInfo shuffleInfo = new ExecutorShuffleInfo(localDirs, 1, "mergedShuffle");
+    pushResolver.registerExecutor(appId, shuffleInfo);
+  }
+
+  private String[] prepareLocalDirs(Path[] localDirs) throws IOException {
+    String[] blockMgrDirs = new String[localDirs.length];
+    for (int i = 0; i< localDirs.length; i++) {
+      Files.createDirectories(localDirs[i].resolve(
+        RemoteBlockPushResolver.MERGE_MANAGER_DIR + File.separator + "00"));
+      blockMgrDirs[i] = localDirs[i].toFile().getPath() + File.separator + BLOCK_MANAGER_DIR;
+    }
+    return blockMgrDirs;
+  }
+
+  private void removeApplication(String appId) {
+    // PushResolver cleans up the local dirs in a different thread which can conflict with the test
+    // data of other tests, since they are using the same Application Id.
+    pushResolver.applicationRemoved(appId,  false);
+  }
+
+  private void validateMergeStatuses(
+      MergeStatuses mergeStatuses,
+      int[] expectedReduceIds,
+      long[] expectedSizes) {
+    assertArrayEquals(expectedReduceIds, mergeStatuses.reduceIds);
+    assertArrayEquals(expectedSizes, mergeStatuses.sizes);
+  }
+
+  private void validateChunks(
+      String appId,
+      int shuffleId,
+      int reduceId,
+      MergedBlockMeta meta,
+      int[] expectedSizes,
+      int[][] expectedMapsPerChunk) throws IOException {
+    assertEquals("num chunks", expectedSizes.length, meta.getNumChunks());
+    RoaringBitmap[] bitmaps = meta.readChunkBitmaps();
+    assertEquals("num of bitmaps", meta.getNumChunks(), bitmaps.length);
+    for (int i = 0; i < meta.getNumChunks(); i++) {
+      RoaringBitmap chunkBitmap = bitmaps[i];
+      Arrays.stream(expectedMapsPerChunk[i]).forEach(x -> assertTrue(chunkBitmap.contains(x)));
+    }
+    for (int i = 0; i < meta.getNumChunks(); i++) {
+      FileSegmentManagedBuffer mb =
+        (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i);
+      assertEquals(expectedSizes[i], mb.getLength());
+    }
+  }
+
+  private void pushBlockHelper(
+      String appId,
+      PushBlock[] blocks,
+      ByteBuffer[] buffers) throws IOException {
+    Preconditions.checkArgument(blocks.length == buffers.length);
+    for (int i = 0; i < blocks.length; i++) {
+      StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
+        new PushBlockStream(appId, blocks[i].shuffleId, blocks[i].mapIndex, blocks[i].reduceId, 0));
+      stream.onData(stream.getID(), buffers[i]);
+      stream.onComplete(stream.getID());
+    }
+  }
+
+  private static class PushBlock {
+    private final int shuffleId;
+    private final int mapIndex;
+    private final int reduceId;

Review comment:
       Done




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r519062524



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       @Ngone51 @attilapiros @Victsm @mridulm I have updated the PR with these changes. Please take a look when you get a chance.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721859821


   **[Test build #130610 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130610/testReport)** for PR 30062 at commit [`7f8303a`](https://github.com/apache/spark/commit/7f8303ab5a3fda52e4748646867cb82088135bdf).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723362017






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723363675






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709810752






----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514027473



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());

Review comment:
       That is a good point @Victsm, the only concern with maintaining a second data structure is potential for divergence (or need to wrap use of either in critical section).
   
   Thoughts @Ngone51 ?




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709600499


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34462/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718995505






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722095287






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719139395


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35031/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516884181



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.streamId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final PushBlockStream msg;
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        PushBlockStream msg,
+        AppShuffleId appShuffleId,
+        int reduceId,
+        int mapIndex,
+        AppShufflePartitionInfo partitionInfo) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.msg = Preconditions.checkNotNull(msg);
+      this.appShuffleId = appShuffleId;
+      this.reduceId = reduceId;
+      this.mapIndex = mapIndex;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+    }
+
+    @Override
+    public String getID() {
+      return msg.streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+            updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", msg.streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;

Review comment:
       These are currently logged during the `onFailure` callback but we are logging at `debug` level.
   Logging them at warning level will clutter the NM logs.
   Also, this seems to be expected behavior. There would be some blocks which didn't get the opportunity to merge and doesn't indicate that something is wrong with the shuffle server which is unexpected. 
   @Victsm What do you think?
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714877548






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714856427






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723562182


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35361/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516912542



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);

Review comment:
       I moved streamId calculation to `receiveBlockDataAsStream` because both the `PushBlockStreamCallback` and the anonymous callback handler need the streamId.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513821005



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());

Review comment:
       One more aspect to consider.
   In order to support YARN NM restart, we need to store the information tracked inside the `partition` map into a levelDB state store, similarly to what `ExternalShuffleBlockResolver` is doing.
   Having 1-level map would make storing the information inside levelDB much easier compared with a 2-level map.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510590175



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513658695



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       > But there the second component is the map ID (with long type) and not the map index (int type) and those two has very different meanings.
   
   Yeah, that part I agree with. In the next patch for the shuffle write path change, where block push gets initiated, we are right now reusing the current `ShuffleBlockId` by passing an int mapIndex to the long mapId field, which is probably hacky.
   I think in addition to making the `PushBlockStream` RPC changes so we are parsing the blockId String inside `OneForOneBlockPusher`, we should also create a new block ID type representing this type of blockId that corresponds to what ShuffleBlockId used to be.
   Any suggestions for the name of this type of block?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509980252



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),
+      Paths.get("target/l2").toAbsolutePath().toString()};
+    cleanupLocalDirs();
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH);
+  }
+
+  @After
+  public void after() {
+    try {
+      cleanupLocalDirs();
+    } catch (IOException e) {
+      // don't fail if clean up doesn't succeed.
+      log.warn("Error deleting test local dirs", e);
+    }
+  }
+
+  private void cleanupLocalDirs() throws IOException {
+    for (String local : localDirs) {
+      FileUtils.deleteDirectory(new File(local));
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      registerApplication(TEST_APP, TEST_USER);
+      registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+      pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+      removeApplication(TEST_APP);
+    } catch (Throwable t) {
+      assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found"));
+      Throwables.propagate(t);
+    }
+  }
+
+  @Test
+  public void testBasicBlockMerge() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+    };
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, blocks);
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}});
+    removeApplication(TEST_APP);
+  }
+
+  @Test
+  public void testDividingMergedBlocksIntoChunks() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_3_0", 0),
+    };
+    ByteBuffer[] buffers = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[2]),
+      ByteBuffer.wrap(new byte[3]),
+      ByteBuffer.wrap(new byte[5]),
+      ByteBuffer.wrap(new byte[3])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, buffers);
+    MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}});
+    removeApplication(TEST_APP);
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));

Review comment:
       Maybe, add `spy()` to `stream2`. So we can verify `writeAnyDeferredBlocks` is really invoked later?

##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),
+      Paths.get("target/l2").toAbsolutePath().toString()};
+    cleanupLocalDirs();
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH);
+  }
+
+  @After
+  public void after() {
+    try {
+      cleanupLocalDirs();
+    } catch (IOException e) {
+      // don't fail if clean up doesn't succeed.
+      log.warn("Error deleting test local dirs", e);
+    }
+  }
+
+  private void cleanupLocalDirs() throws IOException {
+    for (String local : localDirs) {
+      FileUtils.deleteDirectory(new File(local));
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      registerApplication(TEST_APP, TEST_USER);
+      registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+      pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+      removeApplication(TEST_APP);
+    } catch (Throwable t) {
+      assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found"));
+      Throwables.propagate(t);
+    }
+  }
+
+  @Test
+  public void testBasicBlockMerge() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+    };
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, blocks);
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}});
+    removeApplication(TEST_APP);
+  }
+
+  @Test
+  public void testDividingMergedBlocksIntoChunks() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_3_0", 0),
+    };
+    ByteBuffer[] buffers = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[2]),
+      ByteBuffer.wrap(new byte[3]),
+      ByteBuffer.wrap(new byte[5]),
+      ByteBuffer.wrap(new byte[3])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, buffers);
+    MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}});
+    removeApplication(TEST_APP);
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));

Review comment:
       Maybe, uses `spy()` on `stream2`. So we can verify `writeAnyDeferredBlocks` is really invoked later?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509936433



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {

Review comment:
       Is this also used by test only or it will be used in following PRs?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513819945



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());

Review comment:
       Another effect of breaking the map into a 2-level map.
   After we delete the first level map from `partitions` during merge finalization, if there is another block received for the same shuffle, we would then repopulate the first level map with an empty map, even though we have already finalized this shuffle.
   It's possible that we would keep these empty maps in the partitions map until the application finishes.
   
   I feel that breaking the previous 1-level map into a 2-level map has quite some side effects.
   If what we want is to speedup the iteration during shuffle finalization, would keep a separate map for (appId, shuffleId) -> reduceId also address the need?
   This way, we do not break the `partitions` map into a 2-level map which lead to these side effects.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721389401


   **[Test build #130576 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130576/testReport)** for PR 30062 at commit [`27e0d31`](https://github.com/apache/spark/commit/27e0d31a0e2a7350edcf7c388a44ab73d0cdc5fa).


----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723554961


   ok to test


----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514599059



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =

Review comment:
       The concern about race can be addressed by rely on `compute` on the outer map - if we want to ensure atomicity.
   It results in widening the lock per application+shuffle though




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509926188



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -94,6 +95,9 @@
   static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure";
   private static final boolean DEFAULT_STOP_ON_FAILURE = false;
 
+  // Used by shuffle merge manager to create merged shuffle files.
+  protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/";

Review comment:
       Wait, is it the Yarn-specific local dir format? If so, please ignore my above comment.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514622774



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,961 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new StreamCallbackImpl(this, msg, appShuffleId, reduceId, mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.blockId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      logger.info("No partitions to finalize for shuffle {} from Application {}.",
+        msg.shuffleId, msg.appId);
+      return new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    }
+    Collection<AppShufflePartitionInfo> partitions = shufflePartitions.values();
+    int totalPartitions = partitions.size();
+    RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+    int[] reduceIds = new int[totalPartitions];
+    long[] sizes = new long[totalPartitions];
+    Iterator<AppShufflePartitionInfo> partitionsIter = partitions.iterator();
+    int idx = 0;
+    while (partitionsIter.hasNext()) {
+      AppShufflePartitionInfo partition = partitionsIter.next();
+      partitionsIter.remove();

Review comment:
       Deletion should be inside the lock.
   We only delete when we can finalize, and we can only finalize when we acquire the lock.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709607531


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34462/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723543864






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517673813



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.

Review comment:
       I see. spark-annotations is already added as a test dependency for this module. Making the scope compile should be fine then.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719155222






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510531650



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),
+      Paths.get("target/l2").toAbsolutePath().toString()};
+    cleanupLocalDirs();
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH);
+  }
+
+  @After
+  public void after() {
+    try {
+      cleanupLocalDirs();
+    } catch (IOException e) {
+      // don't fail if clean up doesn't succeed.
+      log.warn("Error deleting test local dirs", e);
+    }
+  }
+
+  private void cleanupLocalDirs() throws IOException {
+    for (String local : localDirs) {
+      FileUtils.deleteDirectory(new File(local));
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      registerApplication(TEST_APP, TEST_USER);

Review comment:
       This is done




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709587876


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r518518905



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       That will calculate the length of the bitmap twice every time the chunk is serialized. Also, I don't think that Encoders hold an assumption that the `buf` cannot be expanded. If the type of buf allows expansion it would. If you look at the encode of ByteArrays, even that uses `buf.writeBytes(...)` which will call `ensureWritable(length)`. The difference here is that we changed it to use `RoaringBitmap.serialize` instead of `AbstractByteBuf.writeBytes` so we have to call it explicitly.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509951677



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapId = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapId() {
+      return currentMapId;
+    }
+
+    void setCurrentMapId(int mapId) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}",
+        partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId);
+      this.currentMapId = mapId;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapId) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapId {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, mapId);
+      mapTracker.add(mapId);
+      chunkTracker.add(mapId);
+      lastMergedMapId = mapId;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the mapId to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapId the mapId to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapId) throws IOException {

Review comment:
       nit: rename `mapId` to `mapIndex` and the param doc.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapId = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapId() {
+      return currentMapId;
+    }
+
+    void setCurrentMapId(int mapId) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}",
+        partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId);
+      this.currentMapId = mapId;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapId) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapId {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, mapId);
+      mapTracker.add(mapId);
+      chunkTracker.add(mapId);
+      lastMergedMapId = mapId;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the mapId to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapId the mapId to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapId) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapId);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          partitionId.appId, partitionId.shuffleId, partitionId.reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} reset index to position {}", dataFile.getName(), idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapId) throws IOException {

Review comment:
       nit: rename `mapId` to `mapIndex`




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723540557


   **[Test build #130751 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130751/testReport)** for PR 30062 at commit [`cb1881c`](https://github.com/apache/spark/commit/cb1881cc02e9606471d0f29345267bf2052f6880).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723543864


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514003670



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719437387


   **[Test build #130446 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130446/testReport)** for PR 30062 at commit [`a658f3a`](https://github.com/apache/spark/commit/a658f3acee33be5083c3847a0afa91ac2b82cca4).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  public static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager `
     * `  static class PushBlockStreamCallback implements StreamCallbackWithID `


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721905606


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35213/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516960321



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,39 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * Class name of the implementation of MergedShuffleFileManager that merges the blocks
+   * pushed to it when push-based shuffle is enabled. By default, push-based shuffle is disabled at
+   * a cluster level because this configuration is set to
+   * 'org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager'.
+   * To turn on push-based shuffle at a cluster level, set the configuration to
+   * 'org.apache.spark.network.shuffle.RemoteBlockPushResolver'.
+   */
+  public String mergedShuffleFileManagerImpl() {
+    return conf.get("spark.shuffle.push.based.mergedShuffleFileManagerImpl",

Review comment:
       I have renamed this configuration to `spark.shuffle.server.mergedShuffleFileManagerImpl`. All the server side configs are prefixed with `spark.shuffle.server` and the names do indicate if they are used for merging shuffle blocks.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509888618



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       Let's take an example with 2 blocks: block1 is `shuffle_1_1_1`  and block2 is `shuffle_1_2_1`. The shuffle service should try to append both of these  to `mergedShuffle_1_1.data`. 
   
   When the shuffle service  starts receiving data for `block1` in a stream, let's say this is `stream1`, it appends the received `buf` which is part of `shuffle_1_1_1` to `mergedShuffle_1_1.data` and sets currentMapId = 1. Now before more data is received with `stream1`, the shuffle service receives data for `block2` with `stream2`. When `onData` of `stream2` is invoked, the `buf` will be deferred. Any subsequent `onData` on `stream2` will keep deferring the `bufs`.
   
   The worst case happens when the `onComplete` of `stream2` is invoked before `onComplete` of `stream1`.
   When this happens `stream2` cannot merge the deferred data  to `mergedShuffle_1_1.data` irrespective of where it saves the `deferredBufs`. Right now it's saving it in memory. However, even if it is saving the `deferredBufs` in file, it cannot append it to the `mergedShuffle_1_1.data` because `stream1` hasn't completed yet. 




----------------------------------------------------------------
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.

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


[GitHub] [spark] dongjoon-hyun edited a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun edited a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725733508


   Hi, Guys.
   This seems to break `master` branch with `Hadoop 2.7`.
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/1061/
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/1061/testReport/junit/org.apache.spark.network.yarn/YarnShuffleServiceSuite/testCreateDefaultMergedShuffleFileManagerInstance/history/
   
   ![Screen Shot 2020-11-11 at 4 05 29 PM](https://user-images.githubusercontent.com/9700541/98878396-bc8b9e80-2437-11eb-8dc4-82ebf1b62735.png)
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719404767






----------------------------------------------------------------
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.

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


[GitHub] [spark] dongjoon-hyun edited a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun edited a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725733508


   Hi, Guys.
   This seems to break `master` branch with `Hadoop 2.7`.
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/1061/testReport/junit/org.apache.spark.network.yarn/YarnShuffleServiceSuite/testCreateDefaultMergedShuffleFileManagerInstance/history/
   
   ![Screen Shot 2020-11-11 at 4 05 29 PM](https://user-images.githubusercontent.com/9700541/98878396-bc8b9e80-2437-11eb-8dc4-82ebf1b62735.png)
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722095287






----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-724064024


   my last round of comments were addressed so +1 for me


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723562188






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721594294


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35196/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511022192



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));
+          }
+        }
+        startBlockWrite = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (startBlockWrite && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {

Review comment:
       I am working on replacing `partitions` with `Map[(application_id, shuffleId) -> Map [reduceId -> AppShufflePartitionInfo]`




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512285083



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       `BlockId` is part of the core module and is not available in the network-shuffle module. The `ExternalBlockHandler` also parses the block ids like this.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721896475






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513206585



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.

Review comment:
       Done




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718899685


   **[Test build #130416 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130416/testReport)** for PR 30062 at commit [`5994e4d`](https://github.com/apache/spark/commit/5994e4d322a9b9a89ed6b36ca53cffe5c3524306).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513206659



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714877548






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719212377


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130430/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723543840


   **[Test build #130751 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130751/testReport)** for PR 30062 at commit [`cb1881c`](https://github.com/apache/spark/commit/cb1881cc02e9606471d0f29345267bf2052f6880).
    * This patch **fails due to an unknown error code, -9**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709581026


   **[Test build #129856 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129856/testReport)** for PR 30062 at commit [`f34935f`](https://github.com/apache/spark/commit/f34935f911563f2bca7e151f8429a1a339fa2137).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510667503



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       We do want to reduce the small random reads by the shuffle service. The shuffle server is a shared resource serving shuffle data for multiple apps and these small random reads impacts its performance severely. Please refer to the  push-based shuffle SPIP [doc](https://docs.google.com/document/d/1mYzKVZllA5Flw8AtoX7JUcXBOnNIDADWRbJ7GI6Y71Q/edit) which mentions this as a goal. 
   An executor reading the block again from the file would be much better than increasing the random reads on the shuffle server.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721414299


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35177/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r518548353



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       > Sorry, which two?
   
   1.  `trackerBuf.ensureWritable(Encoders.Bitmaps.encodedLength(chunkTracker)` will call `b.serializedSizeInBytes()`
   2. `Encoders.Bitmaps.encode` will call `b.serializedSizeInBytes()` again.
   
   I am not sure how expensive `RoaringBitmap.serializedSizeInBytes` is, but my preference is to avoid computing that twice.
   
   > I think it's different. First, the invocation of ensureWritable() of ByteArrays is not decided by Spark itself but Netty.
   
   But the API of Spark Encoders  uses `io.netty.buffer.ByteBuf` and doesn't restrict it to a non-expandable ByteBuf implementation.  
   For all the other types we rely on netty's method of writing. This is the only place where we used `RoaringBitmap.serialize`. This is different from the rest of encoders. If adding `ensureWritable` here helps with not calculating the length multiple times, then why not? 




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718995505






----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512886244



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -94,6 +95,9 @@
   static final String STOP_ON_FAILURE_KEY = "spark.yarn.shuffle.stopOnFailure";
   private static final boolean DEFAULT_STOP_ON_FAILURE = false;
 
+  // Used by shuffle merge manager to create merged shuffle files.
+  protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/";

Review comment:
       This goes back to the conversation we had in #29855 (https://github.com/apache/spark/pull/29855/files#r497077157)
   
   The assumption right now is that we can use appId and user Id to properly distinguish the local dirs given different cluster schedulers.
   In the case of YARN, this information is provided through the following API during application registration:
   https://github.com/apache/spark/blob/f284218dae23bf91e72e221943188cdb85e13dac/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java#L58-L66
   
   During executor registration, the `blockManagerDir` local dir paths information is further provided to `RemoteBlockPushResolver` through the following API:
   https://github.com/apache/spark/blob/f284218dae23bf91e72e221943188cdb85e13dac/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java#L68-L77
   Since block merge dirs are per-application instead of per-executor, we only pick the local dir information for the first executor of a given application registered with the shuffle service.
   
   Having the common dir path pattern provided by `YarnShuffleService` to `RemoteBlockPushResolver` would then allow `RemoteBlockPushResolver` to construct the local dirs used for storing merged shuffle files from the `blockManagerDir` local dirs provided during executor registration.
   
   As @otterc mentioned, we don't really need the common dir path pattern if we just store the full local dir paths in the hashmap.
   This would also simplify the application registration API in `MergeShuffleFileManager` to remove the user ID field, which could make it more applicable to other cluster schedulers.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709711121


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34474/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513807702



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =

Review comment:
       I think converting this map into a 2-level map breaks the atomic guarantee we need.
   Inside `onData`, `onComplete`, and `onFailure`, this 2-level map is read.
   However, this read is now broken into 2 non-atomic read:
   You first performs an atomic read of the first level map, followed by an atomic read of the 2nd level map.
   However, together these 2 reads are not atomic.
   Here, when you delete the data from this 2 level map, once again it gets broken into 2 separate deletes, which together is not atomic.
   This would make things go wrong.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714835991


   **[Test build #130182 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130182/testReport)** for PR 30062 at commit [`dfbfc8b`](https://github.com/apache/spark/commit/dfbfc8b2424af41b3238f92ffa7c1f2eb0248197).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712522937






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-717730159


   **[Test build #130357 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130357/testReport)** for PR 30062 at commit [`67cfa83`](https://github.com/apache/spark/commit/67cfa83f31ec8c921f0d1a7291f3189299bf484e).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719132949


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35031/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721440342






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511674349



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =

Review comment:
       @Ngone51  @mridulm 
   I have changed the `partitions` map to `(appId, shuffleId) -> map(reduceId -> appShufflePartitionInfo)`




----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511094756



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the
+   * clients. Instead of serving the entire merged file, the shuffle service serves the
+   * merged file in `chunks`. A `chunk` constitutes few shuffle blocks in entirety and this
+   * configuration controls how big a chunk can get. A corresponding index file for each merged
+   * shuffle file will be generated indicating chunk boundaries.
+   */
+  public int minChunkSizeInMergedShuffleFile() {
+    return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+      conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m")));

Review comment:
       Thanks for making the comment here as I was just wondering the same thing.
   How much difference have you seen in tuning this parameter?   I assume if its not a static here for the server, then it would have to be passed in as a parameter during fetching.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712536768


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130026/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512298080



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile

Review comment:
       Yes, it comes later with the fetch side changes.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r508169611



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));
+          }
+        }
+        startBlockWrite = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (startBlockWrite && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {

Review comment:
       I understand what it does here. But I'm thinking it might be simpler if we could have that map. At least, we don't need to traverse the whole `partitions` to find all the reducer partitions belonging to a certain partition. We can just get them by `map.get(shuffleId)`. The time complexity here is O(n) vs O(1).




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509953273



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);

Review comment:
       Maybe, rename to `dataFile` as we use `indeFile` below?




----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517488389



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX,
+      appShuffleId.shuffleId, msg.mapIndex, msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",

Review comment:
       The case for collisions are not tested. 
   
   Had you considered storing the `deferredBufs` of the detected collisions (this side) and write them out right after the collided callback (the other) finished  its writing?
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r519148908



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,944 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    private DataOutputStream metaWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream indexOutStream = new FileOutputStream(indexFile);
+      this.indexChannel = indexOutStream.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(indexOutStream));

Review comment:
       I have seen the closing of the channel is removed. I do not think it is need but you can simply this part as these two (the FileChannel and the DataOutputStream) can be replaced by a [RandomAccessFile]( https://docs.oracle.com/javase/8/docs/api/java/io/RandomAccessFile.html) (both seek and write is supported).
   
   And you ca do the same for `metaChannel` and `metaWriteStream`.
   
   What do you think?




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723362017






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723349047


   **[Test build #130743 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130743/testReport)** for PR 30062 at commit [`28edaae`](https://github.com/apache/spark/commit/28edaaef5a840716a1a05b3cf99b679e4d27f91f).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723562191


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35361/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725858686


   Thanks for the ping @dongjoon-hyun ! I had not noticed this issue.
   
   @otterc Is it a dependency issue in hadoop-2.7 ?
   
   ```
   [ERROR] testCreateDefaultMergedShuffleFileManagerInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0.627 s  <<< ERROR!
   java.lang.NoClassDefFoundError: org/apache/commons/logging/LogFactory
   	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37)
   Caused by: java.lang.ClassNotFoundException: org.apache.commons.logging.LogFactory
   	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateDefaultMergedShuffleFileManagerInstance(YarnShuffleServiceSuite.java:37)
   
   [ERROR] testCreateRemoteBlockPushResolverInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0 s  <<< ERROR!
   java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService
   	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testCreateRemoteBlockPushResolverInstance(YarnShuffleServiceSuite.java:47)
   
   [ERROR] testInvalidClassNameOfMergeManagerWillUseNoOpInstance(org.apache.spark.network.yarn.YarnShuffleServiceSuite)  Time elapsed: 0.001 s  <<< ERROR!
   java.lang.NoClassDefFoundError: Could not initialize class org.apache.spark.network.yarn.YarnShuffleService
   	at org.apache.spark.network.yarn.YarnShuffleServiceSuite.testInvalidClassNameOfMergeManagerWillUseNoOpInstance(YarnShuffleServiceSuite.java:57)
   ```
   @otterc Can you try building and running master locally with the hadoop-2.7 profile to check please ? That should help understand what the issue is, and more quickly iterate towards a solution.
   I am guessing it should be addition of commons logging for hadoop-2.7 profile for spark network module - but please check. Thanks !


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-720251110






----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513833540



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =

Review comment:
       Inside `onFailure`, access to the map should be inside the lock.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719142535






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [Spark 32916][Shuffle] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709570519


   cc @Victsm @mridulm @tgravescs @jiangxb1987 @attilapiros @Ngone51  
   Please take a look.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723559580


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35361/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-720242467






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721401673


   I am resolving all the older comments which have been addressed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709711130






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709607544






----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709691739


   Can you fix the lint errors in `RemoteBlockPushResolverSuite` @otterc ?


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719199209






----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513428469



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {

Review comment:
       I think we should still remove `partition` from the `shufflePartitions`. Otherwise, `partitionInfoBeforeCheck` can be wrong.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513669307



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       @tgravescs 
   
   > It seems like it would have fairly small impact if you keep it to just the PUshblockStream message and the creation and consumption of it, but maybe I'm missing something?
   
   The impact is small, limited to just where it is created (inside `ShuffleWriter`) and where it consumed.
   I now agree that it's better to make this change, especially given that the block fetching API has made a similar change to parse the String into concrete types on the client side, so the server side does not need to parse.
   
   > the other question is in the prepareBlockPushRequests should it really be using ShuffleBlockId, seems like it would be easy to just create a push based version of the block id but I would have to look more at the full version to see what all that impacts.
   
   Indeed, reusing `ShuffleBlockId` inside `ShuffleWriter#prepareBlockPushRequests` is a bit hacky.
   Any suggestions for the name of this push based version of the shuffle block id?




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513182393



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -279,6 +287,7 @@ public void initializeApplication(ApplicationInitializationContext context) {
     } catch (Exception e) {
       logger.error("Exception when initializing application {}", appId, e);
     }
+    shuffleMergeManager.registerApplication(appId, context.getUser());

Review comment:
       This is a mistake. My next commit will have the fix for this.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-717737324


   I have updated the PR with these changes:
   - Added support for subdirs under merge directory.
   - Removed the yarn local directory path pattern.
   - Removed the `registerApplication` API from `MergedShuffleFileManager` and updated the `registerExecutor` API of `MergedShuffleFileManager`.
   - Addressed few other comments.
   
   These are some pending tasks:
   - Server side configuration which is a class that provides the implementation of push-based shuffle service. Suggested by @tgravescs 
   - Changes to the message `PushBlockStream` as suggested by @attilapiros . This is a bigger change that will touch upon `OneForOneBlockPusher` and `ExternalShuffleBlockClient`.
   - Other review comments from @mridulm and @attilapiros 
   
   @jiangxb1987 @Ngone51 Just checking to see if you have additional feedback on this PR?
   
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-722033650


   **[Test build #130626 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130626/testReport)** for PR 30062 at commit [`a8dd6f5`](https://github.com/apache/spark/commit/a8dd6f58fe65db34770ac4165192188fe3b98639).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719107693


   **[Test build #130423 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130423/testReport)** for PR 30062 at commit [`ed9a697`](https://github.com/apache/spark/commit/ed9a697a28374ab13e4f0f759f00c5fa8a02d615).


----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r518494559



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       ```suggestion
         trackerBuf.ensureWritable(Encoders.Bitmaps.encodedLength(chunkTracker))ï¼›
   ```
   
   hmm...shall we call `ensureWritable` here? I think the `buf` inside `encode()` is always supposed to be capable. We'd better not break the assumption. @otterc 




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719135318


   **[Test build #130430 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130430/testReport)** for PR 30062 at commit [`5b488d7`](https://github.com/apache/spark/commit/5b488d714c4376fc5a8877285ace931645c234b9).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513180854



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,462 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),
+      Paths.get("target/l2").toAbsolutePath().toString()};
+    cleanupLocalDirs();
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH);
+    registerApplication(TEST_APP, TEST_USER);
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+  }
+
+  @After
+  public void after() {
+    try {
+      cleanupLocalDirs();
+      removeApplication(TEST_APP);
+    } catch (Exception e) {
+      // don't fail if clean up doesn't succeed.
+      log.debug("Error while tearing down", e);
+    }
+  }
+
+  private void cleanupLocalDirs() throws IOException {
+    for (String local : localDirs) {
+      FileUtils.deleteDirectory(new File(local));
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    } catch (Throwable t) {
+      assertTrue(t.getMessage().startsWith("Merged shuffle index file"));
+      Throwables.propagate(t);
+    }
+  }
+
+  @Test
+  public void testBasicBlockMerge() throws IOException {
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+    };
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, blocks);
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDividingMergedBlocksIntoChunks() throws IOException {
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_3_0", 0),
+    };
+    ByteBuffer[] buffers = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[2]),
+      ByteBuffer.wrap(new byte[3]),
+      ByteBuffer.wrap(new byte[5]),
+      ByteBuffer.wrap(new byte[3])
+    };
+    pushBlockHelper(TEST_APP, pushBlocks, buffers);
+    MergedBlockMeta meta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}});
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnData() throws IOException {
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    // stream 2 has more data and then completes
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    stream2.onComplete(stream2.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException {
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be deferred
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3]));
+
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    // stream 2 now completes completes
+    stream2.onComplete(stream2.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4, 6}, new int[][]{{0}, {1}});
+  }
+
+  @Test
+  public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOException {
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be ignored
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onComplete(stream2.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
+  }
+
+  @Test
+  public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOException {
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // This should be ignored
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2]));
+
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    // stream 2 now completes completes
+    stream2.onComplete(stream2.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}});
+  }
+
+  @Test
+  public void testFailureAfterData() throws IOException {
+    PushBlockStream pushBlock = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pushBlock.blockId, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    assertEquals("num-chunks", 0, blockMeta.getNumChunks());
+  }
+
+  @Test
+  public void testFailureAfterMultipleDataBlocks() throws IOException {
+    PushBlockStream pushBlock = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pushBlock.blockId, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    assertEquals("num-chunks", 0, blockMeta.getNumChunks());
+  }
+
+  @Test
+  public void testFailureAfterComplete() throws IOException {
+    PushBlockStream pushBlock = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pushBlock.blockId, 0));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[2]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[3]));
+    stream.onData(stream.getID(), ByteBuffer.wrap(new byte[4]));
+    stream.onComplete(stream.getID());
+    stream.onFailure(stream.getID(), new RuntimeException("Forced Failure"));
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
+  }
+
+  @Test (expected = RuntimeException.class)
+  public void testTooLateArrival() throws IOException {
+    PushBlockStream[] pushBlocks = new PushBlockStream[] {
+      new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0),
+      new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0)};
+    ByteBuffer[] blocks = new ByteBuffer[]{
+      ByteBuffer.wrap(new byte[4]),
+      ByteBuffer.wrap(new byte[5])
+    };
+    StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
+      new PushBlockStream(TEST_APP, pushBlocks[0].blockId, 0));
+    for (ByteBuffer block : blocks) {
+      stream.onData(stream.getID(), block);
+    }
+    stream.onComplete(stream.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(
+      new PushBlockStream(TEST_APP, pushBlocks[1].blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4]));
+    try {
+      stream1.onComplete(stream1.getID());
+    } catch (RuntimeException re) {
+      assertEquals(
+        "Block shuffle_0_1_0 received after merged shuffle is finalized", re.getMessage());
+      MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+      validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{9}, new int[][]{{0}});
+      throw re;
+    }
+  }
+
+  @Test
+  public void testIncompleteStreamsAreOverwritten() throws IOException {
+    registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[4]));
+    // There is a failure
+    stream1.onFailure(stream1.getID(), new RuntimeException("forced error"));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[5]));
+    stream2.onComplete(stream2.getID());
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5}, new int[][]{{1}});
+  }
+
+  @Test (expected = RuntimeException.class)
+  public void testFailureWith3Streams() throws IOException {
+    PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0);
+    StreamCallbackWithID stream1 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0));
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+
+    PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0);
+    StreamCallbackWithID stream2 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0));
+    // There is a failure
+    stream2.onFailure(stream2.getID(), new RuntimeException("forced error"));
+
+    PushBlockStream pbStream3 = new PushBlockStream(TEST_APP, "shuffle_0_2_0", 0);
+    StreamCallbackWithID stream3 =
+      pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream3.blockId, 0));
+    // This should be deferred
+    stream3.onData(stream3.getID(), ByteBuffer.wrap(new byte[5]));
+    // Since this stream didn't get any opportunity it will throw couldn't find opportunity error
+    RuntimeException failedEx = null;
+    try {
+      stream3.onComplete(stream2.getID());
+    } catch (RuntimeException re) {
+      assertEquals(
+        "Couldn't find an opportunity to write block shuffle_0_2_0 to merged shuffle",
+        re.getMessage());
+      failedEx = re;
+    }
+    // stream 1 now completes
+    stream1.onData(stream1.getID(), ByteBuffer.wrap(new byte[2]));
+    stream1.onComplete(stream1.getID());
+
+    pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0));
+    MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+    validateChunks(TEST_APP, 0, 0, blockMeta, new int[] {4}, new int[][] {{0}});
+    if (failedEx != null) {
+      throw failedEx;
+    }
+  }
+
+  @Test(expected = NullPointerException.class)
+  public void testUpdateLocalDirsOnlyOnce() throws IOException {
+    String testApp = "updateLocalDirsOnlyOnceTest";
+    registerApplication(testApp, TEST_USER);
+    String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new);
+    registerExecutor(testApp, prepareBlockManagerLocalDirs(testApp, TEST_USER, activeLocalDirs));
+    assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
+    assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
+      "l2/usercache/" + TEST_USER + "/appcache/" + testApp + "/merge_manager"));
+    // Any later app init or executor register from the same application
+    // won't change the active local dirs list
+    registerApplication(testApp, TEST_USER);
+    assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
+    assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
+      "l2/usercache/" + TEST_USER + "/appcache/" + testApp + "/merge_manager"));
+    activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new);
+    registerExecutor(testApp, prepareBlockManagerLocalDirs(testApp, TEST_USER, activeLocalDirs));
+    assertEquals(pushResolver.getMergedBlockDirs(testApp).length, 1);
+    assertTrue(pushResolver.getMergedBlockDirs(testApp)[0].contains(
+      "l2/usercache/" + TEST_USER + "/appcache/" + testApp + "/merge_manager"));
+    removeApplication(testApp);
+    try {
+      pushResolver.getMergedBlockDirs(testApp);
+    } catch (Throwable e) {
+      assertTrue(e.getMessage()
+        .startsWith("application " + testApp + " is not registered or NM was restarted."));
+      Throwables.propagate(e);
+    }
+  }
+
+  /**
+   * Registers the app with RemoteBlockPushResolver.
+   */
+  private void registerApplication(String appId, String user) throws IOException {
+    pushResolver.registerApplication(appId, user);
+  }
+
+  private void registerExecutor(String appId, String[] localDirs) throws IOException {
+    pushResolver.registerExecutor(appId, localDirs);
+    for (String localDir : pushResolver.getMergedBlockDirs(appId)) {
+      Files.createDirectories(Paths.get(localDir));
+    }
+  }
+
+  private String[] prepareBlockManagerLocalDirs(String appId, String user, String[] localDirs){
+    return Arrays.stream(localDirs)
+      .map(localDir ->
+        localDir + "/" + String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, appId))
+      .toArray(String[]::new);
+  }
+
+  private void removeApplication(String appId) {
+    // PushResolver cleans up the local dirs in a different thread which can conflict with the test
+    // data of other tests, since they are using the same Application Id.
+    pushResolver.applicationRemoved(appId,  false);
+  }
+
+  private void validateChunks(
+      String appId,
+      int shuffleId,
+      int reduceId,
+      MergedBlockMeta meta,
+      int[] expectedSizes,
+      int[][] expectedMapsPerChunk) throws IOException {
+    assertEquals("num chunks", expectedSizes.length, meta.getNumChunks());
+    RoaringBitmap[] bitmaps = meta.readChunkBitmaps();
+    assertEquals("num of bitmaps", meta.getNumChunks(), bitmaps.length);
+    for (int i = 0; i < meta.getNumChunks(); i++) {
+      RoaringBitmap chunkBitmap = bitmaps[i];
+      Arrays.stream(expectedMapsPerChunk[i]).forEach(x -> assertTrue(chunkBitmap.contains(x)));
+    }
+    for (int i = 0; i < meta.getNumChunks(); i++) {
+      FileSegmentManagedBuffer mb =
+        (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i);
+      assertEquals(expectedSizes[i], mb.getLength());
+    }
+  }
+
+  private void pushBlockHelper(
+      String appId,
+      PushBlockStream[] pushBlocks,
+      ByteBuffer[] blocks) throws IOException {
+    Preconditions.checkArgument(pushBlocks.length == blocks.length);
+    for (int i = 0; i < pushBlocks.length; i++) {
+      StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream(
+        new PushBlockStream(appId, pushBlocks[i].blockId, 0));

Review comment:
       I'll just pass String[] since appId is also pass in and the index remains same for all the tests right now.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r518992676



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.streamId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final PushBlockStream msg;
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        PushBlockStream msg,
+        AppShuffleId appShuffleId,
+        int reduceId,
+        int mapIndex,
+        AppShufflePartitionInfo partitionInfo) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.msg = Preconditions.checkNotNull(msg);
+      this.appShuffleId = appShuffleId;
+      this.reduceId = reduceId;
+      this.mapIndex = mapIndex;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+    }
+
+    @Override
+    public String getID() {
+      return msg.streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+            updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", msg.streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;

Review comment:
       I'd prefer to get this information through metrics instead of log.
   Since this is inside shuffle service log not Spark driver/executor log, it's usually more difficult to access these logs.
   Server side metrics would help to surface the necessary information without polluting the NM logs with many exceptions that doesn't really hurt.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725868760


   I think there is something different with how the `network-yarn` module is build. I see that there existed 
   `resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala`. 
   I think I shouldn't have added `YarnShuffleServiceSuite.java` in the `network-yarn` module.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719438168


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719120136


   **[Test build #130426 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130426/testReport)** for PR 30062 at commit [`9aee460`](https://github.com/apache/spark/commit/9aee460037cb78784d9c533c5624aba2d5ef04a9).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709810752






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719126738


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35027/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721566228


   **[Test build #130595 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130595/testReport)** for PR 30062 at commit [`7cf38c4`](https://github.com/apache/spark/commit/7cf38c4ad1fd3f62c46b0ff3f9b48490b281085c).


----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511095790



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the
+   * clients. Instead of serving the entire merged file, the shuffle service serves the
+   * merged file in `chunks`. A `chunk` constitutes few shuffle blocks in entirety and this
+   * configuration controls how big a chunk can get. A corresponding index file for each merged
+   * shuffle file will be generated indicating chunk boundaries.
+   */
+  public int minChunkSizeInMergedShuffleFile() {
+    return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+      conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m")));
+  }
+
+  /**
+   * The size of cache used in push-based shuffle for storing merged index files.

Review comment:
       we should specify if cache is memory or disk. 

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,899 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(String.format(
+        "Merged shuffle index file %s of %s not found", indexFile.getPath(), id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle meta file %s of %s not found",
+        metaFile.getPath(), id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(id);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s of %s not found",
+        dataFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s of %s", indexFile.getPath(), id.toString()),
+          e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart

Review comment:
       is there a follow on lira to do this?  what is exact behavior on restart then?

##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -172,7 +178,9 @@ protected void serviceInit(Configuration conf) throws Exception {
       }
 
       TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf));
-      blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile);
+      shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH);

Review comment:
       yeah I think that is the question - are there any side affects that I don't want if I'm not using it. we are registering applications and executors with it even if not in use. I assume that will at least use a bit more memory.  Personally I think its safer to have a way to totally turn it off, I think we could leave that config not published, though really I think all of them are that way.  This also allows someone to turn it off in case they don't want clients to be able to do it at all.  I was also wondering if someone might want to have their own custom implementation for his. So if we made it configurable such that they could provide a class, that would also be supported.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509933833



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {

Review comment:
       If it's used for test only, please add a comment to it, e.g., "used for test only"




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723356780


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35351/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719199209






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721439586


   **[Test build #130576 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130576/testReport)** for PR 30062 at commit [`27e0d31`](https://github.com/apache/spark/commit/27e0d31a0e2a7350edcf7c388a44ab73d0cdc5fa).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516991423



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.

Review comment:
       @mridulm @tgravescs @Ngone51 I have create a follow-up jira for this:
   https://issues.apache.org/jira/browse/SPARK-33331




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719404767


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719212371






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723347195


   **[Test build #130742 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130742/testReport)** for PR 30062 at commit [`be45bf0`](https://github.com/apache/spark/commit/be45bf0cba5c45ccfed5a8bafc91cca1bdffc866).


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719126735






----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714849668


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517831144



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,966 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(
+        this, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.streamId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final PushBlockStream msg;
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        PushBlockStream msg,
+        AppShuffleId appShuffleId,
+        int reduceId,
+        int mapIndex,
+        AppShufflePartitionInfo partitionInfo) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.msg = Preconditions.checkNotNull(msg);
+      this.appShuffleId = appShuffleId;
+      this.reduceId = reduceId;
+      this.mapIndex = mapIndex;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+    }
+
+    @Override
+    public String getID() {
+      return msg.streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+            updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+            appShuffleId.shuffleId, reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", msg.streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;

Review comment:
       I see. I know this's the expected behavior but I really care about this case and want to have a way to easily monitor it.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512259560



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.

Review comment:
       Mostly copied from the method comment of `ExternalShuffleBlockResolver#deleteExecutorDirs`.
   It is indeed serially, since the deletion is handled by a single thread executor.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511675412



##########
File path: common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java
##########
@@ -172,7 +178,9 @@ protected void serviceInit(Configuration conf) throws Exception {
       }
 
       TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf));
-      blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile);
+      shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH);

Review comment:
       @Ngone51 , @tgravescs , @Victsm 
   I will add a configuration on the server side that will disable push-based shuffle on the server.
   
   @tgravescs Do you think the custom implementation can be a follow-up task? I have create this jira https://issues.apache.org/jira/browse/SPARK-33235 for all the follow-up tasks. I will add the custom implementation as a subtask as well.




----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513498062



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       ah yeah the blockId isn't reachable and I missed the whole map index vs id thing. I agree that adding them as separate variables in PushBlockStream seems much more straight forward unless its used by something that needs a block id. Here it doesn't appear to be the case.  would it have a large impact on followup prs as well?   Generally since this is public api would be best to fix before merging.  It seems like it would have fairly small impact if you keep it to just the PUshblockStream message and the creation and consumption of it, but maybe I'm missing something?
   the other question is in the prepareBlockPushRequests should it really be using ShuffleBlockId, seems like it would be easy to just create a push based version of the block id but I would have to look more at the full version to see what all that impacts.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partition.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            try {
+              partition.closeAllFiles();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partition.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public final FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    private final DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      dataFile.createNewFile();
+      this.dataChannel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }
+      Encoders.Bitmaps.encode(trackerBuf, chunkTracker);
+      long metaStartPos = metaChannel.position();
+      try {
+        logger.trace("{} shuffleId {} reduceId {} mapIndex {} write chunk to meta file",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex);
+        metaChannel.write(trackerBuf.nioBuffer());
+      } catch (IOException ioe) {
+        logger.warn("{} shuffleId {} reduceId {} mapIndex {} reset position of meta file to {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex, metaStartPos);
+        metaChannel.position(metaStartPos);
+        throw ioe;
+      } finally {
+        trackerBuf.clear();
+      }
+    }
+
+    void closeAllFiles() throws IOException {
+      this.dataChannel.close();
+      this.metaChannel.close();
+      this.indexWriteStream.close();
+      this.indexChannel.close();
+    }
+  }
+
+  /**
+   * Wraps all the information related to the merge directory of an application.
+   */
+  private static class AppPathsInfo {
+
+    private String[] activeLocalDirs;
+    private int subDirsPerLocalDir;
+
+    private AppPathsInfo updateActiveLocalDirs(
+        String appId,
+        String[] localDirs,
+        int subDirsPerLocalDir) {
+      if (activeLocalDirs == null) {
+        activeLocalDirs = Arrays.stream(localDirs)
+          .map(localDir ->
+            Paths.get(localDir).getParent().resolve(MERGE_MANAGER_DIR).toFile().getPath())

Review comment:
       Just to be clear, we changed this to get the yarn local dir which is format like:
   /hadoop_loc/local/usercache/tgraves/appcache/application_1603128018386_1258
   and then we get the parent which essentially gives us the path we had before usercache/%s/appcache/%s.  I think its worth a comment here to say we are relying on that directory structure and getting parent is ok.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714849668






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714831899


   **[Test build #130181 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130181/testReport)** for PR 30062 at commit [`1ae6852`](https://github.com/apache/spark/commit/1ae6852f47486dda9d1a1b65cd47c2604f6a703a).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709587476


   +CC @venkata91 


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709706624


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34474/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r507462453



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the

Review comment:
       I'm not sure if you're aware of the feature that Spark can save the shuffle data into the disk when the data is too large to hold in the memory.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517495365



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX,
+      appShuffleId.shuffleId, msg.mapIndex, msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",

Review comment:
       There is a previous conversation related to this [here](https://github.com/apache/spark/pull/30062#discussion_r507403306).
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514506049



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,961 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+    if (partitionInfo != null) {
+      return new StreamCallbackImpl(this, msg, appShuffleId, reduceId, mapIndex, partitionInfo);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return msg.blockId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      logger.info("No partitions to finalize for shuffle {} from Application {}.",
+        msg.shuffleId, msg.appId);
+      return new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    }
+    Collection<AppShufflePartitionInfo> partitions = shufflePartitions.values();
+    int totalPartitions = partitions.size();
+    RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+    int[] reduceIds = new int[totalPartitions];
+    long[] sizes = new long[totalPartitions];
+    Iterator<AppShufflePartitionInfo> partitionsIter = partitions.iterator();
+    int idx = 0;
+    while (partitionsIter.hasNext()) {
+      AppShufflePartitionInfo partition = partitionsIter.next();
+      partitionsIter.remove();
+      synchronized (partition) {
+        // Get rid of any partial block data at the end of the file. This could either
+        // be due to failure or a request still being processed when the shuffle
+        // merge gets finalized.
+        try {
+          partition.dataChannel.truncate(partition.getPosition());
+          if (partition.getPosition() != partition.getLastChunkOffset()) {
+            partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+          }
+          bitmaps[idx] = partition.mapTracker;
+          reduceIds[idx] = partition.reduceId;
+          sizes[idx] = partition.getPosition();
+        } catch (IOException ioe) {
+          logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+            msg.shuffleId, partition.reduceId, ioe);
+        } finally {
+          partition.closeAllFiles();
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo());
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs,
+        executorInfo.subDirsPerLocalDir);
+    });
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class StreamCallbackImpl implements StreamCallbackWithID {

Review comment:
       Nit: StreamCallbackImpl -> PushBlockStreamCallback




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510532357



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapId = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapId() {
+      return currentMapId;
+    }
+
+    void setCurrentMapId(int mapId) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}",
+        partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId);
+      this.currentMapId = mapId;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapId) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapId {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, mapId);
+      mapTracker.add(mapId);
+      chunkTracker.add(mapId);
+      lastMergedMapId = mapId;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the mapId to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapId the mapId to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapId) throws IOException {

Review comment:
       Done




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714849656


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34783/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] dongjoon-hyun edited a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun edited a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-725733508


   Hi, Guys.
   This seems to break `master` branch with `Hadoop 2.7`.
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3/969/testReport/junit/org.apache.spark.network.yarn/YarnShuffleServiceSuite/testCreateDefaultMergedShuffleFileManagerInstance/
   - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-hive-2.3-jdk-11/1061/testReport/junit/org.apache.spark.network.yarn/YarnShuffleServiceSuite/testCreateDefaultMergedShuffleFileManagerInstance/history/
   
   ![Screen Shot 2020-11-11 at 4 05 29 PM](https://user-images.githubusercontent.com/9700541/98878396-bc8b9e80-2437-11eb-8dc4-82ebf1b62735.png)
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509977133



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),
+      Paths.get("target/l2").toAbsolutePath().toString()};
+    cleanupLocalDirs();
+    MapConfigProvider provider = new MapConfigProvider(
+      ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4"));
+    conf = new TransportConf("shuffle", provider);
+    pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH);
+  }
+
+  @After
+  public void after() {
+    try {
+      cleanupLocalDirs();
+    } catch (IOException e) {
+      // don't fail if clean up doesn't succeed.
+      log.warn("Error deleting test local dirs", e);
+    }
+  }
+
+  private void cleanupLocalDirs() throws IOException {
+    for (String local : localDirs) {
+      FileUtils.deleteDirectory(new File(local));
+    }
+  }
+
+  @Test(expected = RuntimeException.class)
+  public void testNoIndexFile() {
+    try {
+      registerApplication(TEST_APP, TEST_USER);
+      registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs));
+      pushResolver.getMergedBlockMeta(TEST_APP, 0, 0);
+      removeApplication(TEST_APP);

Review comment:
       Add to `after()`?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r507474727



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);

Review comment:
       I see. Then, shall we rename it to `mapIndex` or `partitionId`?




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709711130






----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r508852242



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       If a stream is `allowedToWrite`, it does write the `deferredBufs` to the merged file. It's line 551-553.
   It's only when the stream is `completing` and still doesn't get a chance then it lands here. Even if we write these deferred blocks to separate files, they cannot be merged to the main data file because some other stream is in between writing. Merging can only happen during `onComplete` when the stream is `allowedToWrite`. If it is not, then it cannot merge.
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r508173722



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);

Review comment:
       I see. Could you add a comment?

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);

Review comment:
       I see. Could you add a comment?




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721404842


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35177/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510577811



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));

Review comment:
       > You can imagine that it's not the stream1 that increases the time to merge, but it's the time combines of stream1 merging time and stream2 merging time
   
   Hmmm. There is additional IO now. stream1 will be reading the data of stream2 from disk which adds considerable time.
   
   Also, the main reason behind push-based shuffle is to convert small random reads to larger sequential reads. The approach of writing individual blocks to disk and then later reading them again will result in a larger random reads.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r519035130



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       I will be incorporating this suggestion when writing the `chunkTracker` to file. This also avoids using an additional bytebuf when writing to the file. 
   I also discussed with @mridulm and @Victsm that whenever we encode a RoaringBitmap in a message `RoaringBitmap.serializedSizeInBytes` is called twice per bitmap. Based on @mridulm suggestion, will update `Encoders.Bitmap.encode`  to 
   ```
       public static void encode(ByteBuf buf, RoaringBitmap b) {
         int length = buf.writableBytes();
         // RoaringBitmap requires nio ByteBuffer for serde. We expose the netty ByteBuf as a nio
         // ByteBuffer. Here, we need to explicitly manage the index so we can write into the
         // ByteBuffer, and the write is reflected in the underneath ByteBuf.
         ByteBuffer byteBuffer = buf.nioBuffer(buf.writerIndex(), length);
         b.serialize(byteBuffer);
         buf.writerIndex(buf.writerIndex() + byteBuffer.position());
       }
   ```




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712536767


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r516175694



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -23,24 +23,31 @@
 import org.apache.spark.network.protocol.Encoders;
 
 // Needed by ScalaDoc. See SPARK-7726
-import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
 
 
 /**
  * Request to push a block to a remote shuffle service to be merged in push based shuffle.
  * The remote shuffle service will also include this message when responding the push requests.
  */
 public class PushBlockStream extends BlockTransferMessage {
+  public static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
   public final String appId;
-  public final String blockId;
+  public final int shuffleId;
+  public final int mapIndex;
+  public final int reduceId;
   // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of
   // blocks to be pushed.
   public final int index;
+  public final String streamId;

Review comment:
       The `streamId` is used for `StreamCallbackWithId.getID()` . The usage of the name `streamId` in `toString` and `decode` is a typo. 

##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,38 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * Class name of the implementation of MergedShuffleFileManager that merges the blocks
+   * pushed to it when push-based shuffle is enabled. Default implementation for merging the blocks
+   * remotely is 'org.apache.spark.network.shuffle.RemoteBlockPushResolver'.
+   * To turn off push-based shuffle at a cluster level, set the configuration to
+   * 'org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager'.
+   */
+  public String mergeShuffleFileManagerImpl() {
+    return conf.get("spark.shuffle.push.based.mergedShuffleFileManagerImpl",
+      "org.apache.spark.network.shuffle.RemoteBlockPushResolver");

Review comment:
       Sure. I will change the default to be `NoOpMergedShuffleFileManager`. 
   cc. @Ngone51 @tgravescs 

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java
##########
@@ -23,24 +23,31 @@
 import org.apache.spark.network.protocol.Encoders;
 
 // Needed by ScalaDoc. See SPARK-7726
-import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type;
 
 
 /**
  * Request to push a block to a remote shuffle service to be merged in push based shuffle.
  * The remote shuffle service will also include this message when responding the push requests.
  */
 public class PushBlockStream extends BlockTransferMessage {
+  public static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
   public final String appId;
-  public final String blockId;
+  public final int shuffleId;
+  public final int mapIndex;
+  public final int reduceId;
   // Similar to the chunkIndex in StreamChunkId, indicating the index of a block in a batch of
   // blocks to be pushed.
   public final int index;
+  public final String streamId;

Review comment:
       This is fixed.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());

Review comment:
       I have made this change

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,883 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =

Review comment:
       There were some race conditions introduced when changed the map into a 2-level map. However, now all of them are addressed thanks to everyone's input. So, I don't think we need to introduce a wider lock. Please take another look at the current changes.




----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512238046



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       this seems out of place to me to do the parsing here.  Can we use the BlockId.apply function to create the type it is and then parse use some parsing functions from the blockid?




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719126735


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723372658


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/130743/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723363675






----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-712536396


   **[Test build #130026 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130026/testReport)** for PR 30062 at commit [`fbdd333`](https://github.com/apache/spark/commit/fbdd33385083adb4be83adf46cd518d519650307).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-717757050


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34960/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513661743



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();

Review comment:
       This is a miss in the earlier implementation.
   The `append = true` is not necessary since we will not append into a merged shuffle file that's already finalized.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714877345


   **[Test build #130181 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130181/testReport)** for PR 30062 at commit [`1ae6852`](https://github.com/apache/spark/commit/1ae6852f47486dda9d1a1b65cd47c2604f6a703a).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511673022



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,899 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(String.format(
+        "Merged shuffle index file %s of %s not found", indexFile.getPath(), id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle meta file %s of %s not found",
+        metaFile.getPath(), id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(id);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s of %s not found",
+        dataFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s of %s", indexFile.getPath(), id.toString()),
+          e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until

Review comment:
       done

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,899 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(String.format(
+        "Merged shuffle index file %s of %s not found", indexFile.getPath(), id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle meta file %s of %s not found",
+        metaFile.getPath(), id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File dataFile = getMergedShuffleDataFile(id);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s of %s not found",
+        dataFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s of %s", indexFile.getPath(), id.toString()),
+          e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", partitionId.appId,
+        partitionId.shuffleId, partitionId.reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.

Review comment:
       done

##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the
+   * clients. Instead of serving the entire merged file, the shuffle service serves the
+   * merged file in `chunks`. A `chunk` constitutes few shuffle blocks in entirety and this
+   * configuration controls how big a chunk can get. A corresponding index file for each merged
+   * shuffle file will be generated indicating chunk boundaries.
+   */
+  public int minChunkSizeInMergedShuffleFile() {
+    return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+      conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m")));
+  }
+
+  /**
+   * The size of cache used in push-based shuffle for storing merged index files.

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719158845


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35035/
   Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719158836






----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512684642



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       What about storing the shuffle ID, **map ID** (and not mapIndex as at line 302), reduce ID in separate member variables within the `PushBlockStream`? 
   And of course with their real types: 
   - shuffle ID: int
   - map ID: long (and not int as in the line 302, you can check this at [ShuffleBlockId](https://github.com/apache/spark/blob/4b6847fdd5491f7858290468f718d033f54e4ff8/core/src/main/scala/org/apache/spark/storage/BlockId.scala#L58)])
   - reduce ID: int
   
   This way the parsing errors are detected at the `PushBlockStream` construction which is much closer to the root cause than this point. 




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719142535






----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513623773



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.

Review comment:
       This would be the max concurrent block streams received by the server that are experiencing collisions.
   It shouldn't be a lot for the following reason:
   To have a large number of concurrent block streams experiencing collisions, the number of shuffle partitions for a given shuffle should be large. However, if that's the case, the randomization on the client side before pushing the blocks would significantly reduce the likelihood of collision in the first place.




----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins commented on pull request #30062: [Spark 32916][Shuffle] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709572516


   Can one of the admins verify this patch?


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517493411



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush";
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX,
+      appShuffleId.shuffleId, msg.mapIndex, msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",

Review comment:
       There is a `testFailureWith3Streams` which tests collisions
   
   > Had you considered storing the deferredBufs of the detected collisions (this side) and write them out right after the collided callback (the other) finished its writing?
   
   I don't understand the suggestion copletely. The deferredBufs are part of the stream and if the stream doesn't get opportunity to write, then it lands here. We are not storing `deferredBufs` outside the stream. 




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513253008



##########
File path: common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java
##########
@@ -363,4 +363,26 @@ public boolean useOldFetchProtocol() {
     return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false);
   }
 
+  /**
+   * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during
+   * push-based shuffle.
+   * A merged shuffle file consists of multiple small shuffle blocks. Fetching the
+   * complete merged shuffle file in a single response increases the memory requirements for the
+   * clients. Instead of serving the entire merged file, the shuffle service serves the
+   * merged file in `chunks`. A `chunk` constitutes few shuffle blocks in entirety and this
+   * configuration controls how big a chunk can get. A corresponding index file for each merged
+   * shuffle file will be generated indicating chunk boundaries.
+   */
+  public int minChunkSizeInMergedShuffleFile() {
+    return Ints.checkedCast(JavaUtils.byteStringAsBytes(
+      conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m")));

Review comment:
       +1 to leave on the server side.




----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r508745474



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,915 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+        // Add `spark` prefix because it will run in NM in Yarn mode.
+        NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+        new CacheLoader<File, ShuffleIndexInformation>() {
+          public ShuffleIndexInformation load(File file) throws IOException {
+            return new ShuffleIndexInformation(file);
+          }
+        };
+    indexCache = CacheBuilder.newBuilder()
+        .maximumWeight(conf.mergedIndexCacheSize())
+        .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+        .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(
+      AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File mergedShuffleFile = getMergedShuffleFile(key);
+      File mergedIndexFile = getMergedIndexFile(id);
+      File mergedMetaFile = getMergedMetaFile(id);
+      try {
+        if (mergedShuffleFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile,
+              mergedMetaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+            "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+                + "meta file {}", key, mergedIndexFile.getAbsolutePath(),
+            mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath());
+        throw new RuntimeException(String.format(
+            "Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(
+      String appId,
+      int shuffleId,
+      int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle index file is not found (id=%s)",
+              id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+          String.format("Application merged shuffle meta file is not found (id=%s)",
+              id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+        new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+        "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(
+          String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(),
+              id.toString()));
+    }
+    File indexFile = getMergedIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+          conf,
+          mergedShuffleFile,
+          shuffleIndexRecord.getOffset(),
+          shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+        "Active local dirs list has not been updated by any executor registration");
+    return
+        Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleFile(AppShufflePartitionId id) {
+    String fileName = id.generateFileName();
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedIndexFile(AppShufflePartitionId id) {
+    String indexName = id.generateIndexFileName();
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedMetaFile(AppShufflePartitionId id) {
+    String metaName = id.generateMetaFileName();
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.get(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(
+        appsPathInfo.get(appId).activeLocalDirs,
+        "application " + appId +
+            " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+        .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+        .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(
+        appsPathInfo.remove(appId),
+        "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+          .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+          .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(msg.appId,
+        Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapId = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+        getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+        && partitionInfoBeforeCheck.mapTracker.contains(mapId) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean startBlockWrite = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of
+       * the block data written to file. In case of failure during writing block to file,
+       * we use the information tracked in partitionInfo to overwrite the corrupt block
+       * when writing the new block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+                "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+                partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+                partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition.
+       * At any given point of time, only a single map stream can write it's data to the merged
+       * file. Until this stream is completed, the other streams defer writing. This prevents
+       * corruption of merged data.
+       * This returns whether this stream is the active stream that can write to the merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time.
+       * One of them is going to be the first to set the currentMapId. When that block does
+       * so, it's going to see the currentMapId initially as -1. After it sets the
+       * currentMapId, it's going to write some data to disk, thus increasing the length
+       * counter. The other duplicate block is going to see the currentMapId already set to
+       * its mapId. However, it hasn't written any data yet. If the first block gets written
+       * completely and resets the currentMapId to -1 before the processing for the second
+       * block finishes, we can just check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapId && length == 0)
+            || partitionInfo.mapTracker.contains(mapId);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write.
+       * The stream first writes any deferred block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            startBlockWrite = true;
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapId);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+            partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            startBlockWrite = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapId);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapId);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+                ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+                msg.blockId));
+          }
+        }
+        startBlockWrite = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (startBlockWrite && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+                partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+        partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {

Review comment:
       Since shuffle id will conflict across applications, I want to understand the proposal better @Ngone51. Is it to replace `partitions` with`Map[(application_id, shuffleId) -> Map [reduceId -> AppShufflePartitionInfo] ]` ?
   
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719135498


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35032/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513686572



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.

Review comment:
       But this will be across all applications here, right ?
   I am trying to get a sense of what the potential impact on memory usage here would be (since it is outside of container restrictions like an executor - and given how lazy java is in releasing memory once acquired).




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r511071765



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {

Review comment:
       The merged shuffle partition is different from regular shuffle blocks in the sense that its ownership is at the host level, not at the executor level.
   There is actually some similarity with SPARK-27651.
   When a reducer is fetching merged shuffle partition chunks from its local host, it's similar to fetching node local shuffle blocks in the sense that the reducer needs to know from the local shuffle service about the local dirs for the blocks.
   We thus leverage SPARK-27651 to reuse existing code path to handle this part.




----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r519148908



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,944 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    private DataOutputStream metaWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream indexOutStream = new FileOutputStream(indexFile);
+      this.indexChannel = indexOutStream.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(indexOutStream));

Review comment:
       I have seen the closing of the channel is removed. I do not think it is need but you can simply this part as these two (the `FileChannel` and the `DataOutputStream`) can be replaced by a [RandomAccessFile]( https://docs.oracle.com/javase/8/docs/api/java/io/RandomAccessFile.html) (both seek and write is supported).
   
   And you can do the same for `metaChannel` and `metaWriteStream`.
   
   What do you think?




----------------------------------------------------------------
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.

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


[GitHub] [spark] Ngone51 commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Ngone51 commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r509945735



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapId = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);

Review comment:
       Can't we simply do `indexWriteStream.writeLong(0L)` here? I prefer to get rid of the special case of `mapId=-1` here.




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513206538



##########
File path: common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java
##########
@@ -0,0 +1,462 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+
+import com.google.common.base.Preconditions;
+import com.google.common.base.Throwables;
+import com.google.common.collect.ImmutableMap;
+
+import org.apache.commons.io.FileUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.MapConfigProvider;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * Tests for {@link RemoteBlockPushResolver}.
+ */
+public class RemoteBlockPushResolverSuite {
+
+  private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class);
+  private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/";
+  private final String TEST_USER = "testUser";
+  private final String TEST_APP = "testApp";
+  private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401";
+
+  private TransportConf conf;
+  private RemoteBlockPushResolver pushResolver;
+  private String[] localDirs;
+
+  @Before
+  public void before() throws IOException {
+    localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(),

Review comment:
       Done




----------------------------------------------------------------
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.

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


[GitHub] [spark] mridulm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513686572



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.

Review comment:
       But this will be across all applications here, right ?
   I am trying to get a sense of what the potential impact on memory usage here would be in degenerate situations (since it is outside of container restrictions like an executor - and given how lazy java is in releasing memory once acquired).




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719158826


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35035/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [Spark 32916][Shuffle] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709569269


   Can one of the admins verify this patch?


----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514454561



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
        I'd just say something like ShufflePushBlockId




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719151559


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35035/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-709692748


   **[Test build #129869 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129869/testReport)** for PR 30062 at commit [`dceae72`](https://github.com/apache/spark/commit/dceae72f4a1719972cda23c8ea9f2309c129c4dd).


----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r514001854



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, Integer.parseInt(blockIdParts[1]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    int reduceId = Integer.parseInt(blockIdParts[3]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(),
+              updatedPos);
+            length += partitionInfo.dataChannel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.dataChannel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write its data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapIndex() < 0
+          || partitionInfo.getCurrentMapIndex() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapIndex. When that block does so, it's
+       * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+       * going to write some data to disk, thus increasing the length counter. The other duplicate
+       * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+       * written any data yet. If the first block gets written completely and resets the
+       * currentMapIndex to -1 before the processing for the second block finishes, we can just
+       * check the bitmap to identify the second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBufs() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map index to make sure only block matching the map index can be written to
+        // disk. If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+          // then it means that the shuffle merge has already been finalized. We should thus ignore
+          // the data and just drain the remaining bytes of this message. This check should be
+          // placed inside the synchronized block to make sure that checking the key is still
+          // present and processing the data is atomic.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              partitionInfo.setCurrentMapIndex(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBufs();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+            // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+            // memory.
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId,
+          appShuffleId.shuffleId, reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapIndex() < 0) {
+              writeAnyDeferredBufs();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapIndex(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+        if (isWriting && partitionInfo != null && shufflePartitions != null
+          && shufflePartitions.containsKey(reduceId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId,
+              appShuffleId.shuffleId, reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId));
+    if (shufflePartitions != null) {
+      for (AppShufflePartitionInfo partition : shufflePartitions.values()) {
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partition.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            try {
+              partition.closeAllFiles();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partition.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),

Review comment:
       done




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-718926290


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35020/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r513642780



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       > This is the same as the the usage of blockId for fetching blocks ...
   
   If it is the same then it would be https://github.com/apache/spark/blob/4b6847fdd5491f7858290468f718d033f54e4ff8/core/src/main/scala/org/apache/spark/storage/BlockId.scala#L58
   
   Right?
   
   But there the second component is the map ID (with long type) and not the map index (int type) and those two has very different meanings.
   




----------------------------------------------------------------
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.

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


[GitHub] [spark] otterc commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r510494220



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,893 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShufflePartitionId, AppShufflePartitionInfo> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given an ID 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(AppShufflePartitionId id) {
+    return partitions.computeIfAbsent(id, 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.
+      File dataFile = getMergedShuffleDataFile(id);
+      File indexFile = getMergedShuffleIndexFile(id);
+      File metaFile = getMergedShuffleMetaFile(id);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(id, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File indexFile = getMergedShuffleIndexFile(id);
+    if (!indexFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle index file is not found (id=%s)", id.toString()));
+    }
+    int size = (int) indexFile.length();
+    // First entry is the zero offset
+    int numChunks = (size / Long.BYTES) - 1;
+    File metaFile = getMergedShuffleMetaFile(id);
+    if (!metaFile.exists()) {
+      throw new RuntimeException(
+        String.format("Application merged shuffle meta file is not found (id=%s)", id.toString()));
+    }
+    FileSegmentManagedBuffer chunkBitMaps =
+      new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length());
+    logger.trace(
+      "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId);
+    File mergedShuffleFile = getMergedShuffleDataFile(id);
+    if (!mergedShuffleFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle file %s of %s not found",
+        mergedShuffleFile.getPath(), id.toString()));
+    }
+    File indexFile = getMergedShuffleIndexFile(id);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException("Failed to open file: " + indexFile, e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShufflePartitionId id) {
+    String fileName = String.format("%s.data", id.generateFileName());
+    return getFile(id.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShufflePartitionId id) {
+    String indexName = String.format("%s.index", id.generateFileName());
+    return getFile(id.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShufflePartitionId id) {
+    String metaName = String.format("%s.meta", id.generateFileName());
+    return getFile(id.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (appId.equals(partitionId.appId)) {
+        iterator.remove();
+        try {
+          partition.channel.close();
+        } catch (IOException e) {
+          logger.error("Error closing merged shuffle file for {}", partitionId);
+        }
+      }
+    }
+
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");
+    if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) {
+      throw new IllegalArgumentException("Unexpected shuffle block id format: " + msg.blockId);
+    }
+    AppShufflePartitionId partitionId = new AppShufflePartitionId(
+      msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3]));
+    int mapIndex = Integer.parseInt(blockIdParts[2]);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(partitionId);
+
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given mapId
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck;
+
+    return new StreamCallbackWithID() {
+      private int length = 0;
+      // This indicates that this stream got the opportunity to write the blocks to the merged file.
+      // Once this is set to true and the stream encounters a failure then it will take necessary
+      // action to overwrite any partial written data. This is reset to false when the stream
+      // completes without any failures.
+      private boolean isWriting = false;
+      // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+      private List<ByteBuffer> deferredBufs;
+
+      @Override
+      public String getID() {
+        return msg.blockId;
+      }
+
+      /**
+       * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+       * block data written to file. In case of failure during writing block to file, we use the
+       * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+       * block.
+       */
+      private void writeBuf(ByteBuffer buf) throws IOException {
+        while (buf.hasRemaining()) {
+          assert partitionInfo != null;
+          if (partitionInfo.isEncounteredFailure()) {
+            long updatedPos = partitionInfo.getPosition() + length;
+            logger.debug(
+              "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+              partitionId.appId, partitionId.shuffleId, partitionId.reduceId,
+              partitionInfo.getPosition(), updatedPos);
+            length += partitionInfo.channel.write(buf, updatedPos);
+          } else {
+            length += partitionInfo.channel.write(buf);
+          }
+        }
+      }
+
+      /**
+       * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+       * given point of time, only a single map stream can write it's data to the merged file. Until
+       * this stream is completed, the other streams defer writing. This prevents corruption of
+       * merged data. This returns whether this stream is the active stream that can write to the
+       * merged file.
+       */
+      private boolean allowedToWrite() {
+        assert partitionInfo != null;
+        return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex;
+      }
+
+      /**
+       * Returns if this is a duplicate block generated by speculative tasks. With speculative
+       * tasks, we could receive the same block from 2 different sources at the same time. One of
+       * them is going to be the first to set the currentMapId. When that block does so, it's going
+       * to see the currentMapId initially as -1. After it sets the currentMapId, it's going to
+       * write some data to disk, thus increasing the length counter. The other duplicate block is
+       * going to see the currentMapId already set to its mapId. However, it hasn't written any data
+       * yet. If the first block gets written completely and resets the currentMapId to -1 before
+       * the processing for the second block finishes, we can just check the bitmap to identify the
+       * second as a duplicate.
+       */
+      private boolean isDuplicateBlock() {
+        assert partitionInfo != null;
+        return (partitionInfo.getCurrentMapId() == mapIndex && length == 0)
+          || partitionInfo.mapTracker.contains(mapIndex);
+      }
+
+      /**
+       * This is only invoked when the stream is able to write. The stream first writes any deferred
+       * block parts buffered in memory.
+       */
+      private void writeAnyDeferredBlocks() throws IOException {
+        assert partitionInfo != null;
+        if (deferredBufs != null && !deferredBufs.isEmpty()) {
+          for (ByteBuffer deferredBuf : deferredBufs) {
+            writeBuf(deferredBuf);
+          }
+          deferredBufs = null;
+        }
+      }
+
+      @Override
+      public void onData(String streamId, ByteBuffer buf) throws IOException {
+        // If partition info is null, ignore the requests. It could only be
+        // null either when a request is received after the shuffle file is
+        // finalized or when a request is for a duplicate block.
+        if (partitionInfo == null) {
+          return;
+        }
+        // When handling the block data using StreamInterceptor, it can help to reduce the amount
+        // of data that needs to be buffered in memory since it does not wait till the completion
+        // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+        // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+        // the benefit of handling the block data using StreamInterceptor as much as possible while
+        // providing the guarantee that one block would be continuously written to the merged
+        // shuffle file before the next block starts. For each shuffle partition, we would track
+        // the current map id to make sure only block matching the map id can be written to disk.
+        // If one server thread sees the block being handled is the current block, it would
+        // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+        // If the block becomes the current block before we see the end of it, we would then dump
+        // all buffered block data to disk and write the remaining portions of the block directly
+        // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+        // memory, while still providing the necessary guarantee.
+        synchronized (partitionInfo) {
+          // If the key is no longer present in the map, it means the shuffle merge has already
+          // been finalized. We should thus ignore the data and just drain the remaining bytes of
+          // this message. This check should be placed inside the synchronized block to make sure
+          // that checking the key is still present and processing the data is atomic.
+          if (!partitions.containsKey(partitionId)) {
+            // TODO is it necessary to dereference deferredBufs?
+            deferredBufs = null;
+            return;
+          }
+          // Check whether we can write to disk
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            if (partitionInfo.getCurrentMapId() < 0) {
+              partitionInfo.setCurrentMapId(mapIndex);
+            }
+
+            // If we got here, it's safe to write the block data to the merged shuffle file. We
+            // first write any deferred block.
+            writeAnyDeferredBlocks();
+            writeBuf(buf);
+            // If we got here, it means we successfully write the current chunk of block to merged
+            // shuffle file. If we encountered failure while writing the previous block, we should
+            // reset the file channel position and the status of partitionInfo to indicate that we
+            // have recovered from previous disk write failure. However, we do not update the
+            // position tracked by partitionInfo here. That is only updated while the entire block
+            // is successfully written to merged shuffle file.
+            if (partitionInfo.isEncounteredFailure()) {
+              partitionInfo.channel.position(partitionInfo.getPosition() + length);
+              partitionInfo.setEncounteredFailure(false);
+            }
+          } else {
+            logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            // If we cannot write to disk, we buffer the current block chunk in memory so it could
+            // potentially be written to disk later. We take our best effort without guarantee
+            // that the block will be written to disk. If the block data is divided into multiple
+            // chunks during TCP transportation, each #onData invocation is an attempt to write
+            // the block to disk. If the block is still not written to disk after all #onData
+            // invocations, the final #onComplete invocation is the last attempt to write the
+            // block to disk. If we still couldn't write this block to disk after this, we give up
+            // on this block push request and respond failure to client. We could potentially
+            // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+            // to increase the chance of writing the block to disk, however this would incur more
+            // memory footprint or decrease the server processing throughput for the shuffle
+            // service. In addition, during test we observed that by randomizing the order in
+            // which clients sends block push requests batches, only ~0.5% blocks failed to be
+            // written to disk due to this reason. We thus decide to optimize for server
+            // throughput and memory usage.
+            if (deferredBufs == null) {
+              deferredBufs = new LinkedList<>();
+            }
+            // Write the buffer to the in-memory deferred cache
+            ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+            deferredBuf.put(buf);
+            deferredBuf.flip();
+            deferredBufs.add(deferredBuf);
+          }
+        }
+      }
+
+      @Override
+      public void onComplete(String streamId) throws IOException {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId,
+          partitionId.shuffleId, partitionId.reduceId);
+        if (partitionInfo == null) {
+          if (isTooLate) {
+            // 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", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          } else {
+            // For duplicate block that is received before the shuffle merge finalizes, the
+            // server should respond success to the client.
+            return;
+          }
+        }
+        // TODO should the merge manager check for the merge completion ratio here and finalize
+        // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+        // TODO and the file channel can be closed even if finalize merge request is somehow not
+        // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+        synchronized (partitionInfo) {
+          // When this request initially got to the server, the shuffle merge finalize request
+          // was not received yet. By the time we finish reading this message, the shuffle merge
+          // however is already finalized. We should thus respond RpcFailure to the client.
+          if (!partitions.containsKey(partitionId)) {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("Block %s %s", msg.blockId,
+              ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // Check if we can commit this block
+          if (allowedToWrite()) {
+            isWriting = true;
+            // Identify duplicate block generated by speculative tasks. We respond success to
+            // the client in cases of duplicate even though no data is written.
+            if (isDuplicateBlock()) {
+              deferredBufs = null;
+              return;
+            }
+            if (partitionInfo.getCurrentMapId() < 0) {
+              writeAnyDeferredBlocks();
+            }
+            long updatedPos = partitionInfo.getPosition() + length;
+            boolean indexUpdated = false;
+            if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) {
+              partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+              indexUpdated = true;
+            }
+            partitionInfo.setPosition(updatedPos);
+            partitionInfo.setCurrentMapId(-1);
+
+            // update merged results
+            partitionInfo.blockMerged(mapIndex);
+            if (indexUpdated) {
+              partitionInfo.resetChunkTracker();
+            }
+          } else {
+            deferredBufs = null;
+            throw new RuntimeException(String.format("%s %s to merged shuffle",
+              ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+              msg.blockId));
+          }
+        }
+        isWriting = false;
+      }
+
+      @Override
+      public void onFailure(String streamId, Throwable throwable) throws IOException {
+        if (errorHandler.shouldLogError(throwable)) {
+          logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        } else {
+          logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable);
+        }
+        // Only update partitionInfo if the failure corresponds to a valid request. If the
+        // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+        // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+        // to write the block data to disk, we should also ignore here.
+        if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) {
+          synchronized (partitionInfo) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId,
+              partitionId.shuffleId, partitionId.reduceId);
+            partitionInfo.setCurrentMapId(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    };
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    List<RoaringBitmap> bitmaps = new LinkedList<>();
+    List<Integer> reduceIds = new LinkedList<>();
+    List<Long> sizes = new LinkedList<>();
+    Iterator<Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShufflePartitionId, AppShufflePartitionInfo> entry = iterator.next();
+      AppShufflePartitionId partitionId = entry.getKey();
+      AppShufflePartitionInfo partition = entry.getValue();
+      if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) {
+        synchronized (partition) {
+          iterator.remove();
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.channel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId);
+            }
+            bitmaps.add(partition.mapTracker);
+            reduceIds.add(partitionId.reduceId);
+            sizes.add(partition.getPosition());
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partitionId.reduceId, ioe);
+          } finally {
+            try {
+              partition.channel.close();
+              partition.metaChannel.close();
+              partition.indexWriteStream.close();
+            } catch (IOException closeEx) {
+              logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId,
+                msg.shuffleId, partitionId.reduceId, closeEx);
+            }
+          }
+        }
+      }
+    }
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return new MergeStatuses(msg.shuffleId, bitmaps.toArray(new RoaringBitmap[bitmaps.size()]),
+      Ints.toArray(reduceIds), Longs.toArray(sizes));
+  }
+
+  @Override
+  public void registerApplication(String appId, String user) {
+    logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user);
+    appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user));
+  }
+
+  @Override
+  public void registerExecutor(String appId, String[] localDirs) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} {}",
+        appId, Arrays.toString(localDirs));
+    }
+    Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> {
+      assert appPathsInfo != null;
+      return appPathsInfo.updateActiveLocalDirs(
+        targetAppId, relativeMergeDirPathPattern, localDirs);
+    });
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle partition for an application. This is used to key the
+   * metadata tracked for each shuffle partition that's being actively merged.
+   */
+  public static class AppShufflePartitionId {
+    public final String appId;
+    public final int shuffleId;
+    public final int reduceId;
+
+    AppShufflePartitionId(String appId, int shuffleId, int reduceId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+      this.reduceId = reduceId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShufflePartitionId that = (AppShufflePartitionId) o;
+      return shuffleId == that.shuffleId && reduceId == that.reduceId
+        && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId, reduceId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .add("reduceId", reduceId)
+        .toString();
+    }
+
+    String generateFileName() {
+      return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId);
+    }
+
+    boolean compareAppShuffleId(String appId, int shuffleId) {
+      return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId;
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShufflePartitionId partitionId;
+    // The merged shuffle data file
+    final File dataFile;
+    public final FileChannel channel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map Id whose block is being merged for this shuffle partition
+    private int currentMapId;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private final FileChannel indexChannel;
+    // The meta file for a particular merged shuffle contains all the map ids that belong to every
+    // chunk. The entry per chunk is a serialized bitmap.
+    private final FileChannel metaChannel;
+    private final DataOutputStream indexWriteStream;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapId = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShufflePartitionId partitionId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.partitionId = Preconditions.checkNotNull(partitionId, "partition id");
+      dataFile.createNewFile();
+      this.dataFile = dataFile;
+      this.channel = new FileOutputStream(dataFile, true).getChannel();
+      indexFile.createNewFile();
+      FileOutputStream fos = new FileOutputStream(indexFile, true);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaFile.createNewFile();
+      metaChannel = new FileOutputStream(metaFile, true).getChannel();
+      this.currentMapId = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);

Review comment:
       Will have to repeat some log statements. If there is an exception just while writing to index file, we do want the logs to reflect it. So, some trivial code is repeated. I do prefer writing to index file from a single method instead of writing it from different locations. I think that helps with debugging.
   Anyways, I don't have a strong opinion about this. If you insist on changing this, I will make that change.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723375340


   **[Test build #130742 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130742/testReport)** for PR 30062 at commit [`be45bf0`](https://github.com/apache/spark/commit/be45bf0cba5c45ccfed5a8bafc91cca1bdffc866).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723372657


   Merged build finished. Test FAILed.


----------------------------------------------------------------
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.

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


[GitHub] [spark] tgravescs commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r517620874



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.

Review comment:
       this and other new classes (including MergedShuffleFileManager) should have the:
    * @since 3.1.0 or @Since("3.1.0")
   depending on java/scala.
   
   Also sorry if  this has been discussed do we want to marke the new API as Evolving?
   
   Note I'm fine with doing this as separate issue.

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.
+
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }
+      Encoders.Bitmaps.encode(trackerBuf, chunkTracker);
+      long metaStartPos = metaChannel.position();
+      try {
+        logger.trace("{} shuffleId {} reduceId {} mapIndex {} write chunk to meta file",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex);
+        metaChannel.write(trackerBuf.nioBuffer());
+      } catch (IOException ioe) {
+        logger.warn("{} shuffleId {} reduceId {} mapIndex {} reset position of meta file to {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, mapIndex, metaStartPos);
+        metaChannel.position(metaStartPos);
+        throw ioe;
+      } finally {
+        trackerBuf.clear();
+      }
+    }
+
+    void closeAllFiles() {
+      try {
+        if (dataChannel != null) {
+          dataChannel.close();
+          dataChannel = null;

Review comment:
       should the setting to null really be in a finally block? That way if the close does throw and we ignore it still gets set

##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          // TODO is it necessary to dereference deferredBufs?
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      // TODO should the merge manager check for the merge completion ratio here and finalize
+      // TODO shuffle merge if appropriate? So the merge manager can potentially finalize early
+      // TODO and the file channel can be closed even if finalize merge request is somehow not
+      // TODO received from the driver? If so, then we need to know # maps for this shuffle.

Review comment:
       what is the plan with the rest of the TODO's in this file?




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-723363667


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35352/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-719196800


   **[Test build #130427 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130427/testReport)** for PR 30062 at commit [`6037d5e`](https://github.com/apache/spark/commit/6037d5e68165edb02962c57e3a55e15914763d5c).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds the following public classes _(experimental)_:
     * `  static class PushBlockStreamCallback implements StreamCallbackWithID `


----------------------------------------------------------------
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.

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


[GitHub] [spark] attilapiros commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
attilapiros commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r512684642



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,905 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  private static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final String relativeMergeDirPathPattern;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.relativeMergeDirPathPattern = relativeMergeDirPathPattern;
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile);
+        }
+      } catch (IOException e) {
+        logger.error(
+          "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and "
+            + "meta file {}", key, indexFile.getAbsolutePath(),
+            indexFile.getAbsolutePath(), metaFile.getAbsolutePath());
+        throw new RuntimeException(
+          String.format("Cannot initialize merged shuffle partition %s", key.toString()), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  // TODO should we use subDirsPerLocalDir to potentially reduce inode size?
+  private File getFile(String appId, String filename) {
+    int hash = JavaUtils.nonNegativeHash(filename);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs);
+    Path localDir = activeLocalDirs[hash % activeLocalDirs.length];
+    String relativePath = getRelativePath(appPathsInfo.user, appId);
+    Path filePath = localDir.resolve(relativePath);
+    File targetFile = new File(filePath.toFile(), filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private Path[] getActiveLocalDirs(String[] activeLocalDirs) {
+    Preconditions.checkNotNull(activeLocalDirs,
+      "Active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs).map(localDir -> Paths.get(localDir)).toArray(Path[]::new);
+  }
+
+  private String getRelativePath(String user, String appId) {
+    return String.format(relativeMergeDirPathPattern + MERGE_MANAGER_DIR, user, appId);
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appsPathInfo.get(appId).activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return Arrays.stream(activeLocalDirs)
+      .map(dir -> dir + getRelativePath(appPathsInfo.user, appId))
+      .toArray(String[]::new);
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          try {
+            partitionInfo.closeAllFiles();
+          } catch (IOException e) {
+            logger.error("Error closing merged shuffle files for appId {} shuffleId {} reduceId {}",
+              appId, partitionInfo.appShuffleId.shuffleId, partitionInfo.reduceId);
+          }
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs))
+        .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId)))
+        .toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Synchronously delete local dirs, executed in a separate thread.
+   */
+  private void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    String[] blockIdParts = msg.blockId.split("_");

Review comment:
       What about storing the shuffle ID, **map ID** (and not mapIndex as at line 302), reduce ID in separate member variables within the `PushBlockStream`? 
   And of course with their real types: 
   - shuffle ID: int
   - map ID: long (and not int as in the line 302, you can check this at [ShuffleBlockId](https://github.com/apache/spark/blob/4b6847fdd5491f7858290468f718d033f54e4ff8/core/src/main/scala/org/apache/spark/storage/BlockId.scala#L58)
   - reduce ID: int
   
   This way the parsing errors are detected at the `PushBlockStream` construction which is much closer to the root cause than this point. 
   
   The map ID identifies the mapper task: see https://github.com/apache/spark/blob/4b6847fdd5491f7858290468f718d033f54e4ff8/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala#L96-L99
   The map index is an index within the map stage (it's the partition ID).
   So it is important not to mix these two.




----------------------------------------------------------------
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.

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


[GitHub] [spark] Victsm commented on a change in pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
Victsm commented on a change in pull request #30062:
URL: https://github.com/apache/spark/pull/30062#discussion_r519033988



##########
File path: common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -0,0 +1,959 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.network.shuffle;
+
+import java.io.BufferedOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executor;
+import java.util.concurrent.Executors;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Preconditions;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.Weigher;
+import com.google.common.collect.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.roaringbitmap.RoaringBitmap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.spark.network.buffer.FileSegmentManagedBuffer;
+import org.apache.spark.network.buffer.ManagedBuffer;
+import org.apache.spark.network.client.StreamCallbackWithID;
+import org.apache.spark.network.protocol.Encoders;
+import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo;
+import org.apache.spark.network.shuffle.protocol.FinalizeShuffleMerge;
+import org.apache.spark.network.shuffle.protocol.MergeStatuses;
+import org.apache.spark.network.shuffle.protocol.PushBlockStream;
+import org.apache.spark.network.util.JavaUtils;
+import org.apache.spark.network.util.NettyUtils;
+import org.apache.spark.network.util.TransportConf;
+
+/**
+ * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle
+ * service processing logic to support push based shuffle.
+ *
+ * @since 3.1.0
+ */
+public class RemoteBlockPushResolver implements MergedShuffleFileManager {
+
+  private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class);
+  @VisibleForTesting
+  static final String MERGE_MANAGER_DIR = "merge_manager";
+
+  private final ConcurrentMap<String, AppPathsInfo> appsPathInfo;
+  private final ConcurrentMap<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> partitions;
+
+  private final Executor directoryCleaner;
+  private final TransportConf conf;
+  private final int minChunkSize;
+  private final ErrorHandler.BlockPushErrorHandler errorHandler;
+
+  @SuppressWarnings("UnstableApiUsage")
+  private final LoadingCache<File, ShuffleIndexInformation> indexCache;
+
+  @SuppressWarnings("UnstableApiUsage")
+  public RemoteBlockPushResolver(TransportConf conf) {
+    this.conf = conf;
+    this.partitions = Maps.newConcurrentMap();
+    this.appsPathInfo = Maps.newConcurrentMap();
+    this.directoryCleaner = Executors.newSingleThreadExecutor(
+      // Add `spark` prefix because it will run in NM in Yarn mode.
+      NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner"));
+    this.minChunkSize = conf.minChunkSizeInMergedShuffleFile();
+    CacheLoader<File, ShuffleIndexInformation> indexCacheLoader =
+      new CacheLoader<File, ShuffleIndexInformation>() {
+        public ShuffleIndexInformation load(File file) throws IOException {
+          return new ShuffleIndexInformation(file);
+        }
+      };
+    indexCache = CacheBuilder.newBuilder()
+      .maximumWeight(conf.mergedIndexCacheSize())
+      .weigher((Weigher<File, ShuffleIndexInformation>) (file, indexInfo) -> indexInfo.getSize())
+      .build(indexCacheLoader);
+    this.errorHandler = new ErrorHandler.BlockPushErrorHandler();
+  }
+
+  /**
+   * Given the appShuffleId 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(
+      AppShuffleId appShuffleId,
+      int reduceId) {
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!partitions.containsKey(appShuffleId) && dataFile.exists()) {
+      // If this partition is already finalized then the partitions map will not contain
+      // the appShuffleId but the data file would exist. In that case the block is considered late.
+      return null;
+    }
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+      partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap());
+    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.
+      File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+      File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId);
+      try {
+        if (dataFile.exists()) {
+          return null;
+        } else {
+          return new AppShufflePartitionInfo(appShuffleId, 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", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e);
+      }
+    });
+  }
+
+  @Override
+  public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, 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 = getMergedShuffleMetaFile(appShuffleId, 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);
+    return new MergedBlockMeta(numChunks, chunkBitMaps);
+  }
+
+  @SuppressWarnings("UnstableApiUsage")
+  @Override
+  public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) {
+    AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId);
+    File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId);
+    if (!dataFile.exists()) {
+      throw new RuntimeException(String.format("Merged shuffle data file %s not found",
+        dataFile.getPath()));
+    }
+    File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId);
+    try {
+      // If we get here, the merged shuffle file should have been properly finalized. Thus we can
+      // use the file length to determine the size of the merged shuffle block.
+      ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile);
+      ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId);
+      return new FileSegmentManagedBuffer(
+        conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength());
+    } catch (ExecutionException e) {
+      throw new RuntimeException(String.format(
+        "Failed to open merged shuffle index file %s", indexFile.getPath()), e);
+    }
+  }
+
+  /**
+   * The logic here is consistent with
+   * org.apache.spark.storage.DiskBlockManager#getMergedShuffleFile
+   */
+  private File getFile(String appId, String filename) {
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs,
+      appPathsInfo.subDirsPerLocalDir, filename);
+    logger.debug("Get merged file {}", targetFile.getAbsolutePath());
+    return targetFile;
+  }
+
+  private File getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) {
+    String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, fileName);
+  }
+
+  private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) {
+    String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, indexName);
+  }
+
+  private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) {
+    String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId));
+    return getFile(appShuffleId.appId, metaName);
+  }
+
+  @Override
+  public String[] getMergedBlockDirs(String appId) {
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.get(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.activeLocalDirs,
+      "application " + appId
+      + " active local dirs list has not been updated by any executor registration");
+    return activeLocalDirs;
+  }
+
+  @Override
+  public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
+    logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs);
+    // TODO: [SPARK-33236] Change the message when this service is able to handle NM restart
+    AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId),
+      "application " + appId + " is not registered or NM was restarted.");
+    Iterator<Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>>> iterator =
+      partitions.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<AppShuffleId, Map<Integer, AppShufflePartitionInfo>> entry = iterator.next();
+      AppShuffleId appShuffleId = entry.getKey();
+      if (appId.equals(appShuffleId.appId)) {
+        iterator.remove();
+        for (AppShufflePartitionInfo partitionInfo : entry.getValue().values()) {
+          partitionInfo.closeAllFiles();
+        }
+      }
+    }
+    if (cleanupLocalDirs) {
+      Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs)
+        .map(dir -> Paths.get(dir)).toArray(Path[]::new);
+      directoryCleaner.execute(() -> deleteExecutorDirs(dirs));
+    }
+  }
+
+  /**
+   * Serially delete local dirs, executed in a separate thread.
+   */
+  @VisibleForTesting
+  void deleteExecutorDirs(Path[] dirs) {
+    for (Path localDir : dirs) {
+      try {
+        if (Files.exists(localDir)) {
+          JavaUtils.deleteRecursively(localDir.toFile());
+          logger.debug("Successfully cleaned up directory: {}", localDir);
+        }
+      } catch (Exception e) {
+        logger.error("Failed to delete directory: {}", localDir, e);
+      }
+    }
+  }
+
+  @Override
+  public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) {
+    // Retrieve merged shuffle file metadata
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    AppShufflePartitionInfo partitionInfoBeforeCheck =
+      getOrCreateAppShufflePartitionInfo(appShuffleId, msg.reduceId);
+    // Here partitionInfo will be null in 2 cases:
+    // 1) The request is received for a block that has already been merged, this is possible due
+    // to the retry logic.
+    // 2) The request is received after the merged shuffle is finalized, thus is too late.
+    //
+    // For case 1, we will drain the data in the channel and just respond success
+    // to the client. This is required because the response of the previously merged
+    // block will be ignored by the client, per the logic in RetryingBlockFetcher.
+    // Note that the netty server should receive data for a given block id only from 1 channel
+    // at any time. The block should be pushed only from successful maps, thus there should be
+    // only 1 source for a given block at any time. Although the netty client might retry sending
+    // this block to the server multiple times, the data of the same block always arrives from the
+    // same channel thus the server should have already processed the previous request of this
+    // block before seeing it again in the channel. This guarantees that we can simply just
+    // check the bitmap to determine if a block is a duplicate or not.
+    //
+    // For case 2, we will also drain the data in the channel, but throw an exception in
+    // {@link org.apache.spark.network.client.StreamCallback#onComplete(String)}. This way,
+    // the client will be notified of the failure but the channel will remain active. Keeping
+    // the channel alive is important because the same channel could be reused by multiple map
+    // tasks in the executor JVM, which belongs to different stages. While one of the shuffles
+    // in these stages is finalized, the others might still be active. Tearing down the channel
+    // on the server side will disrupt these other on-going shuffle merges. It's also important
+    // to notify the client of the failure, so that it can properly halt pushing the remaining
+    // blocks upon receiving such failures to preserve resources on the server/client side.
+    //
+    // Speculative execution would also raise a possible scenario with duplicate blocks. Although
+    // speculative execution would kill the slower task attempt, leading to only 1 task attempt
+    // succeeding in the end, there is no guarantee that only one copy of the block will be
+    // pushed. This is due to our handling of block push process outside of the map task, thus
+    // it is possible for the speculative task attempt to initiate the block push process before
+    // 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 after shuffle merge finalize
+    final boolean isTooLate = partitionInfoBeforeCheck == null;
+    // Check if the given block is already merged by checking the bitmap against the given map index
+    final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null
+      && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null
+        : partitionInfoBeforeCheck;
+    final String streamId = String.format("%s_%d_%d_%d",
+      OneForOneBlockPusher.SHUFFLE_PUSH_BLOCK_PREFIX, appShuffleId.shuffleId, msg.mapIndex,
+      msg.reduceId);
+    if (partitionInfo != null) {
+      return new PushBlockStreamCallback(this, streamId, partitionInfo, msg.mapIndex);
+    } else {
+      // For a duplicate block or a block which is late, respond back with a callback that handles
+      // them differently.
+      return new StreamCallbackWithID() {
+        @Override
+        public String getID() {
+          return streamId;
+        }
+
+        @Override
+        public void onData(String streamId, ByteBuffer buf) {
+          // Ignore the requests. It reaches here either when a request is received after the
+          // shuffle file is finalized or when a request is for a duplicate block.
+        }
+
+        @Override
+        public void onComplete(String streamId) {
+          if (isTooLate) {
+            // 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.TOO_LATE_MESSAGE_SUFFIX));
+          }
+          // For duplicate block that is received before the shuffle merge finalizes, the
+          // server should respond success to the client.
+        }
+
+        @Override
+        public void onFailure(String streamId, Throwable cause) {
+        }
+      };
+    }
+  }
+
+  @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+  @Override
+  public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException {
+    logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId);
+    Map<Integer, AppShufflePartitionInfo> shufflePartitions = partitions.get(appShuffleId);
+    MergeStatuses mergeStatuses;
+    if (shufflePartitions == null || shufflePartitions.isEmpty()) {
+      mergeStatuses =
+        new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]);
+    } else {
+      Collection<AppShufflePartitionInfo> partitionsToFinalize = shufflePartitions.values();
+      int totalPartitions = partitionsToFinalize.size();
+      RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions];
+      int[] reduceIds = new int[totalPartitions];
+      long[] sizes = new long[totalPartitions];
+      Iterator<AppShufflePartitionInfo> partitionsIter = partitionsToFinalize.iterator();
+      int idx = 0;
+      while (partitionsIter.hasNext()) {
+        AppShufflePartitionInfo partition = partitionsIter.next();
+        synchronized (partition) {
+          // Get rid of any partial block data at the end of the file. This could either
+          // be due to failure or a request still being processed when the shuffle
+          // merge gets finalized.
+          try {
+            partition.dataChannel.truncate(partition.getPosition());
+            if (partition.getPosition() != partition.getLastChunkOffset()) {
+              partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex);
+            }
+            bitmaps[idx] = partition.mapTracker;
+            reduceIds[idx] = partition.reduceId;
+            sizes[idx++] = partition.getPosition();
+          } catch (IOException ioe) {
+            logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId,
+              msg.shuffleId, partition.reduceId, ioe);
+          } finally {
+            partition.closeAllFiles();
+            // The partition should be removed after the files are written so that any new stream
+            // for the same reduce partition will see that the data file exists.
+            partitionsIter.remove();
+          }
+        }
+      }
+      mergeStatuses = new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes);
+    }
+    partitions.remove(appShuffleId);
+    logger.info("Finalized shuffle {} from Application {}.", msg.shuffleId, msg.appId);
+    return mergeStatuses;
+  }
+
+  @Override
+  public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) {
+    if (logger.isDebugEnabled()) {
+      logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} "
+        + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs),
+          executorInfo.subDirsPerLocalDir);
+    }
+    appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs,
+      executorInfo.subDirsPerLocalDir));
+  }
+
+  private static String generateFileName(AppShuffleId appShuffleId, int reduceId) {
+    return String.format("mergedShuffle_%s_%d_%d", appShuffleId.appId, appShuffleId.shuffleId,
+      reduceId);
+  }
+
+  /**
+   * Callback for push stream that handles blocks which are not already merged.
+   */
+  static class PushBlockStreamCallback implements StreamCallbackWithID {
+
+    private final RemoteBlockPushResolver mergeManager;
+    private final String streamId;
+    private final int mapIndex;
+    private final AppShufflePartitionInfo partitionInfo;
+    private int length = 0;
+    // This indicates that this stream got the opportunity to write the blocks to the merged file.
+    // Once this is set to true and the stream encounters a failure then it will take necessary
+    // action to overwrite any partial written data. This is reset to false when the stream
+    // completes without any failures.
+    private boolean isWriting = false;
+    // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly
+    private List<ByteBuffer> deferredBufs;
+
+    private PushBlockStreamCallback(
+        RemoteBlockPushResolver mergeManager,
+        String streamId,
+        AppShufflePartitionInfo partitionInfo,
+        int mapIndex) {
+      this.mergeManager = Preconditions.checkNotNull(mergeManager);
+      this.streamId = streamId;
+      this.partitionInfo = Preconditions.checkNotNull(partitionInfo);
+      this.mapIndex = mapIndex;
+    }
+
+    @Override
+    public String getID() {
+      return streamId;
+    }
+
+    /**
+     * Write a ByteBuffer to the merged shuffle file. Here we keep track of the length of the
+     * block data written to file. In case of failure during writing block to file, we use the
+     * information tracked in partitionInfo to overwrite the corrupt block when writing the new
+     * block.
+     */
+    private void writeBuf(ByteBuffer buf) throws IOException {
+      while (buf.hasRemaining()) {
+        if (partitionInfo.isEncounteredFailure()) {
+          long updatedPos = partitionInfo.getPosition() + length;
+          logger.debug(
+            "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos);
+          length += partitionInfo.dataChannel.write(buf, updatedPos);
+        } else {
+          length += partitionInfo.dataChannel.write(buf);
+        }
+      }
+    }
+
+    /**
+     * There will be multiple streams of map blocks belonging to the same reduce partition. At any
+     * given point of time, only a single map stream can write its data to the merged file. Until
+     * this stream is completed, the other streams defer writing. This prevents corruption of
+     * merged data. This returns whether this stream is the active stream that can write to the
+     * merged file.
+     */
+    private boolean allowedToWrite() {
+      return partitionInfo.getCurrentMapIndex() < 0
+        || partitionInfo.getCurrentMapIndex() == mapIndex;
+    }
+
+    /**
+     * Returns if this is a duplicate block generated by speculative tasks. With speculative
+     * tasks, we could receive the same block from 2 different sources at the same time. One of
+     * them is going to be the first to set the currentMapIndex. When that block does so, it's
+     * going to see the currentMapIndex initially as -1. After it sets the currentMapIndex, it's
+     * going to write some data to disk, thus increasing the length counter. The other duplicate
+     * block is going to see the currentMapIndex already set to its mapIndex. However, it hasn't
+     * written any data yet. If the first block gets written completely and resets the
+     * currentMapIndex to -1 before the processing for the second block finishes, we can just
+     * check the bitmap to identify the second as a duplicate.
+     */
+    private boolean isDuplicateBlock() {
+      return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0)
+        || partitionInfo.mapTracker.contains(mapIndex);
+    }
+
+    /**
+     * This is only invoked when the stream is able to write. The stream first writes any deferred
+     * block parts buffered in memory.
+     */
+    private void writeAnyDeferredBufs() throws IOException {
+      if (deferredBufs != null && !deferredBufs.isEmpty()) {
+        for (ByteBuffer deferredBuf : deferredBufs) {
+          writeBuf(deferredBuf);
+        }
+        deferredBufs = null;
+      }
+    }
+
+    @Override
+    public void onData(String streamId, ByteBuffer buf) throws IOException {
+      // When handling the block data using StreamInterceptor, it can help to reduce the amount
+      // of data that needs to be buffered in memory since it does not wait till the completion
+      // of the frame before handling the message, thus releasing the ByteBuf earlier. However,
+      // this also means it would chunk a block into multiple buffers. Here, we want to preserve
+      // the benefit of handling the block data using StreamInterceptor as much as possible while
+      // providing the guarantee that one block would be continuously written to the merged
+      // shuffle file before the next block starts. For each shuffle partition, we would track
+      // the current map index to make sure only block matching the map index can be written to
+      // disk. If one server thread sees the block being handled is the current block, it would
+      // directly write the block to disk. Otherwise, it would buffer the block chunks in memory.
+      // If the block becomes the current block before we see the end of it, we would then dump
+      // all buffered block data to disk and write the remaining portions of the block directly
+      // to disk as well. This way, we avoid having to buffer the entirety of every blocks in
+      // memory, while still providing the necessary guarantee.
+      synchronized (partitionInfo) {
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // If the partitionInfo corresponding to (appId, shuffleId, reduceId) is no longer present
+        // then it means that the shuffle merge has already been finalized. We should thus ignore
+        // the data and just drain the remaining bytes of this message. This check should be
+        // placed inside the synchronized block to make sure that checking the key is still
+        // present and processing the data is atomic.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          return;
+        }
+        // Check whether we can write to disk
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          logger.trace("{} shuffleId {} reduceId {} onData writable",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            partitionInfo.setCurrentMapIndex(mapIndex);
+          }
+
+          // If we got here, it's safe to write the block data to the merged shuffle file. We
+          // first write any deferred block.
+          writeAnyDeferredBufs();
+          writeBuf(buf);
+          // If we got here, it means we successfully write the current chunk of block to merged
+          // shuffle file. If we encountered failure while writing the previous block, we should
+          // reset the file channel position and the status of partitionInfo to indicate that we
+          // have recovered from previous disk write failure. However, we do not update the
+          // position tracked by partitionInfo here. That is only updated while the entire block
+          // is successfully written to merged shuffle file.
+          if (partitionInfo.isEncounteredFailure()) {
+            partitionInfo.dataChannel.position(partitionInfo.getPosition() + length);
+            partitionInfo.setEncounteredFailure(false);
+          }
+        } else {
+          logger.trace("{} shuffleId {} reduceId {} onData deferred",
+            partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+            partitionInfo.reduceId);
+          // If we cannot write to disk, we buffer the current block chunk in memory so it could
+          // potentially be written to disk later. We take our best effort without guarantee
+          // that the block will be written to disk. If the block data is divided into multiple
+          // chunks during TCP transportation, each #onData invocation is an attempt to write
+          // the block to disk. If the block is still not written to disk after all #onData
+          // invocations, the final #onComplete invocation is the last attempt to write the
+          // block to disk. If we still couldn't write this block to disk after this, we give up
+          // on this block push request and respond failure to client. We could potentially
+          // buffer the block longer or wait for a few iterations inside #onData or #onComplete
+          // to increase the chance of writing the block to disk, however this would incur more
+          // memory footprint or decrease the server processing throughput for the shuffle
+          // service. In addition, during test we observed that by randomizing the order in
+          // which clients sends block push requests batches, only ~0.5% blocks failed to be
+          // written to disk due to this reason. We thus decide to optimize for server
+          // throughput and memory usage.
+          if (deferredBufs == null) {
+            deferredBufs = new LinkedList<>();
+          }
+          // Write the buffer to the in-memory deferred cache. Since buf is a slice of a larger
+          // byte buffer, we cache only the relevant bytes not the entire large buffer to save
+          // memory.
+          ByteBuffer deferredBuf = ByteBuffer.allocate(buf.remaining());
+          deferredBuf.put(buf);
+          deferredBuf.flip();
+          deferredBufs.add(deferredBuf);
+        }
+      }
+    }
+
+    @Override
+    public void onComplete(String streamId) throws IOException {
+      synchronized (partitionInfo) {
+        logger.trace("{} shuffleId {} reduceId {} onComplete invoked",
+          partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+          partitionInfo.reduceId);
+        Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+          mergeManager.partitions.get(partitionInfo.appShuffleId);
+        // When this request initially got to the server, the shuffle merge finalize request
+        // was not received yet. By the time we finish reading this message, the shuffle merge
+        // however is already finalized. We should thus respond RpcFailure to the client.
+        if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("Block %s %s", streamId,
+            ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX));
+        }
+        // Check if we can commit this block
+        if (allowedToWrite()) {
+          isWriting = true;
+          // Identify duplicate block generated by speculative tasks. We respond success to
+          // the client in cases of duplicate even though no data is written.
+          if (isDuplicateBlock()) {
+            deferredBufs = null;
+            return;
+          }
+          if (partitionInfo.getCurrentMapIndex() < 0) {
+            writeAnyDeferredBufs();
+          }
+          long updatedPos = partitionInfo.getPosition() + length;
+          boolean indexUpdated = false;
+          if (updatedPos - partitionInfo.getLastChunkOffset() >= mergeManager.minChunkSize) {
+            partitionInfo.updateChunkInfo(updatedPos, mapIndex);
+            indexUpdated = true;
+          }
+          partitionInfo.setPosition(updatedPos);
+          partitionInfo.setCurrentMapIndex(-1);
+
+          // update merged results
+          partitionInfo.blockMerged(mapIndex);
+          if (indexUpdated) {
+            partitionInfo.resetChunkTracker();
+          }
+        } else {
+          deferredBufs = null;
+          throw new RuntimeException(String.format("%s %s to merged shuffle",
+            ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX,
+            streamId));
+        }
+      }
+      isWriting = false;
+    }
+
+    @Override
+    public void onFailure(String streamId, Throwable throwable) throws IOException {
+      if (mergeManager.errorHandler.shouldLogError(throwable)) {
+        logger.error("Encountered issue when merging {}", streamId, throwable);
+      } else {
+        logger.debug("Encountered issue when merging {}", streamId, throwable);
+      }
+      // Only update partitionInfo if the failure corresponds to a valid request. If the
+      // request is too late, i.e. received after shuffle merge finalize, #onFailure will
+      // also be triggered, and we can just ignore. Also, if we couldn't find an opportunity
+      // to write the block data to disk, we should also ignore here.
+      if (isWriting) {
+        synchronized (partitionInfo) {
+          Map<Integer, AppShufflePartitionInfo> shufflePartitions =
+            mergeManager.partitions.get(partitionInfo.appShuffleId);
+          if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) {
+            logger.debug("{} shuffleId {} reduceId {} set encountered failure",
+              partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId,
+              partitionInfo.reduceId);
+            partitionInfo.setCurrentMapIndex(-1);
+            partitionInfo.setEncounteredFailure(true);
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * ID that uniquely identifies a shuffle for an application. This is used as a key in
+   * {@link #partitions}.
+   */
+  public static class AppShuffleId {
+    public final String appId;
+    public final int shuffleId;
+
+    AppShuffleId(String appId, int shuffleId) {
+      this.appId = appId;
+      this.shuffleId = shuffleId;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      AppShuffleId that = (AppShuffleId) o;
+      return shuffleId == that.shuffleId && Objects.equal(appId, that.appId);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hashCode(appId, shuffleId);
+    }
+
+    @Override
+    public String toString() {
+      return Objects.toStringHelper(this)
+        .add("appId", appId)
+        .add("shuffleId", shuffleId)
+        .toString();
+    }
+  }
+
+  /** Metadata tracked for an actively merged shuffle partition */
+  public static class AppShufflePartitionInfo {
+
+    private final AppShuffleId appShuffleId;
+    private final int reduceId;
+    // The merged shuffle data file channel
+    public FileChannel dataChannel;
+    // Location offset of the last successfully merged block for this shuffle partition
+    private long position;
+    // Indicating whether failure was encountered when merging the previous block
+    private boolean encounteredFailure;
+    // Track the map index whose block is being merged for this shuffle partition
+    private int currentMapIndex;
+    // Bitmap tracking which mapper's blocks have been merged for this shuffle partition
+    private RoaringBitmap mapTracker;
+    // The index file for a particular merged shuffle contains the chunk offsets.
+    private FileChannel indexChannel;
+    private DataOutputStream indexWriteStream;
+    // The meta file for a particular merged shuffle contains all the map indices that belong to
+    // every chunk. The entry per chunk is a serialized bitmap.
+    private FileChannel metaChannel;
+    // The offset for the last chunk tracked in the index file for this shuffle partition
+    private long lastChunkOffset;
+    private int lastMergedMapIndex = -1;
+
+    // Bitmap tracking which mapper's blocks are in the current shuffle chunk
+    private RoaringBitmap chunkTracker;
+    ByteBuf trackerBuf = null;
+
+    AppShufflePartitionInfo(
+        AppShuffleId appShuffleId,
+        int reduceId,
+        File dataFile,
+        File indexFile,
+        File metaFile) throws IOException {
+      this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id");
+      this.reduceId = reduceId;
+      this.dataChannel = new FileOutputStream(dataFile).getChannel();
+      FileOutputStream fos = new FileOutputStream(indexFile);
+      indexChannel = fos.getChannel();
+      this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos));
+      metaChannel = new FileOutputStream(metaFile).getChannel();
+      this.currentMapIndex = -1;
+      // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex()
+      updateChunkInfo(0L, -1);
+      this.position = 0;
+      this.encounteredFailure = false;
+      this.mapTracker = new RoaringBitmap();
+      this.chunkTracker = new RoaringBitmap();
+    }
+
+    public long getPosition() {
+      return position;
+    }
+
+    public void setPosition(long position) {
+      logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, this.position, position);
+      this.position = position;
+    }
+
+    boolean isEncounteredFailure() {
+      return encounteredFailure;
+    }
+
+    void setEncounteredFailure(boolean encounteredFailure) {
+      this.encounteredFailure = encounteredFailure;
+    }
+
+    int getCurrentMapIndex() {
+      return currentMapIndex;
+    }
+
+    void setCurrentMapIndex(int mapIndex) {
+      logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}",
+        appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex);
+      this.currentMapIndex = mapIndex;
+    }
+
+    long getLastChunkOffset() {
+      return lastChunkOffset;
+    }
+
+    void blockMerged(int mapIndex) {
+      logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId,
+        appShuffleId.shuffleId, reduceId, mapIndex);
+      mapTracker.add(mapIndex);
+      chunkTracker.add(mapIndex);
+      lastMergedMapIndex = mapIndex;
+    }
+
+    void resetChunkTracker() {
+      chunkTracker.clear();
+    }
+
+    /**
+     * Appends the chunk offset to the index file and adds the map index to the chunk tracker.
+     *
+     * @param chunkOffset the offset of the chunk in the data file.
+     * @param mapIndex the map index to be added to chunk tracker.
+     */
+    void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
+      long idxStartPos = -1;
+      try {
+        // update the chunk tracker to meta file before index file
+        writeChunkTracker(mapIndex);
+        idxStartPos = indexChannel.position();
+        logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}",
+          appShuffleId.appId, appShuffleId.shuffleId, reduceId, this.lastChunkOffset,
+          chunkOffset);
+        indexWriteStream.writeLong(chunkOffset);
+      } catch (IOException ioe) {
+        if (idxStartPos != -1) {
+          // reset the position to avoid corrupting index files during exception.
+          logger.warn("{} shuffleId {} reduceId {} reset index to position {}",
+            appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos);
+          indexChannel.position(idxStartPos);
+        }
+        throw ioe;
+      }
+      this.lastChunkOffset = chunkOffset;
+    }
+
+    private void writeChunkTracker(int mapIndex) throws IOException {
+      if (mapIndex == -1) {
+        return;
+      }
+      chunkTracker.add(mapIndex);
+      if (trackerBuf == null) {
+        trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker));
+      }

Review comment:
       I think @attilapiros's suggestion makes sense.
   We can just add a file based serialization API inside `Encoders.Bitmaps` to consolidate all related logic at one place.
   We do not need a file based deserialization API though.
   The metaFile is transferred from server side to the client side leveraging Netty zero copy transfer, thus the server side does not need to perform any deserialization.
   The client side will receive the data already as Netty ByteBuf, and can use existing `Bitmaps#decode` for deserialization.
   This code is already committed in #29855 
   https://github.com/apache/spark/blob/master/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedBlockMeta.java#L55
   
   By doing this, we can get rid of `trackerBuf` and all the associated issues discussed here.




----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714843730


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/34783/
   


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-714835991


   **[Test build #130182 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130182/testReport)** for PR 30062 at commit [`dfbfc8b`](https://github.com/apache/spark/commit/dfbfc8b2424af41b3238f92ffa7c1f2eb0248197).


----------------------------------------------------------------
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.

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


[GitHub] [spark] SparkQA commented on pull request #30062: [SPARK-32916][SHUFFLE] Implementation of shuffle service that leverages push-based shuffle in YARN deployment mode

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30062:
URL: https://github.com/apache/spark/pull/30062#issuecomment-721928842


   **[Test build #130610 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130610/testReport)** for PR 30062 at commit [`7f8303a`](https://github.com/apache/spark/commit/7f8303ab5a3fda52e4748646867cb82088135bdf).
    * This patch **fails Spark unit tests**.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
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.

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