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:
us...@infra.apache.org



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

Reply via email to