From df48e011372c33a2cd2331267c9eab3dace91a2c Mon Sep 17 00:00:00 2001 From: Min Shen Date: Sat, 9 May 2020 16:08:41 -0700 Subject: [PATCH 01/35] LIHADOOP-48527 Magnet shuffle service block transfer netty protocol The following changes are included in this patch. In addition, fixed a potential block duplicate issue when speculative execution is enabled, and improved test coverage. commit 7e134c2b75c8882474a67c15036087eb8a02caef Author: Chandni Singh Date: Tue Apr 21 21:54:46 2020 -0700 LIHADOOP-52202 Create merge_manager under app local dirs RB=2069854 G=superfriends-reviewers R=mshen,yezhou A=chsingh commit 63dcda9309fc06c5f1fb6e7268df1d7416db49c7 Author: Chandni Singh Date: Wed Apr 1 12:15:21 2020 -0700 LIHADOOP-51889 Divide remote fetches into smaller chunks RB=2029681 commit 35298465155ec10e6ee2caf1adc0e78717dc6fed Author: Chandni Singh Date: Thu Mar 19 17:47:40 2020 -0700 LIHADOOP-51889 Writing last chunk offsets to merge index file RB=2016700 BUG=LIHADOOP-51889 G=superfriends-reviewers R=mshen A=mshen commit bbb53ec0fdfa0aebda954ede17a9f6e217607a53 Author: Min Shen Date: Thu Dec 19 08:46:34 2019 -0800 Shuffle server and client properly handles merged block fetch failure. Use file length as merged shuffle block size when serving merged shuffle block. commit 6718074c6a6a98b1d66d4fdff6bf08fb266ce32e Author: Min Shen Date: Mon Nov 18 14:19:20 2019 -0800 Netty protocol for DAGScheduler control message commit 52e4dfade2e004fbc39fc60937342a9a57872680 Author: Min Shen Date: Sun Sep 8 18:44:09 2019 -0700 Netty protocol for remote block push, pass 3 commit e9db4cc1ae56e9722a598b0011a10e55e84bf19c Author: Min Shen Date: Thu Sep 5 18:29:24 2019 -0700 Netty protocol for remote block push, pass 2 commit 7627ecf54292edda4a133e596f53306e7af76100 Author: Min Shen Date: Fri Aug 30 08:54:08 2019 -0700 Netty protocol for remote block push, pass 1 --- .../shuffle/RemoteBlockPushResolver.java | 685 ++++++++++++++++++ .../shuffle/RemoteBlockPushResolverSuite.java | 152 ++++ .../network/yarn/YarnShuffleService.java | 19 +- 3 files changed, 855 insertions(+), 1 deletion(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java create mode 100644 common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java new file mode 100644 index 000000000000..3b7d0d6148d6 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -0,0 +1,685 @@ +/* + * 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 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.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 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 final Path[] localDirs; + private final ConcurrentMap appsRelativePath; + private final ConcurrentMap partitions; + + private final Executor directoryCleaner; + private final TransportConf conf; + private final int minChunkSize; + + private final LoadingCache indexCache; + + @SuppressWarnings("UnstableApiUsage") + public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { + this.conf = conf; + this.localDirs = new Path[localDirs.length]; + for (int i = 0; i < localDirs.length; i++) { + this.localDirs[i] = Paths.get(localDirs[i]); + } + this.partitions = Maps.newConcurrentMap(); + this.appsRelativePath = 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(); + String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); + CacheLoader indexCacheLoader = + new CacheLoader() { + public ShuffleIndexInformation load(File file) throws IOException { + return new ShuffleIndexInformation(file); + } + }; + indexCache = CacheBuilder.newBuilder() + .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) + .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) + .build(indexCacheLoader); + } + + /** + * 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. + try { + File mergedShuffleFile = getMergedShuffleFile(key); + if (mergedShuffleFile.exists()) { + return null; + } else { + return new AppShufflePartitionInfo(mergedShuffleFile, getMergedIndexFile(id)); + } + } catch (IOException e) { + throw new RuntimeException(String.format( + "Cannot initialize merged shuffle partition %s", key.toString()), e); + } + }); + } + + @Override + public int getChunkCount(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 + return (size / Long.BYTES) - 1; + } + + @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("Application merged shuffle file is not found (id=%s)", 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); + Path localDir = localDirs[hash % localDirs.length]; + Path relativeMergeDir = Preconditions.checkNotNull( + appsRelativePath.get(appId), "application " + appId + " is not registered."); + return new File(localDir.resolve(relativeMergeDir).toFile(), filename); + } + + 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); + } + + @Override + public void applicationRemoved(String appId, boolean cleanupLocalDirs) { + logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); + Path relativeMergeDir = appsRelativePath.remove(appId); + Iterator> iterator = + partitions.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry 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(localDirs) + .map(dir -> dir.resolve(relativeMergeDir)).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; + private boolean canWrite = true; + // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly + private List 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()) { + length += partitionInfo.channel.write(buf, partitionInfo.getPosition() + length); + } else { + length += partitionInfo.channel.write(buf); + } + } + } + + @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 (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { + // Check 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. + if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || + partitionInfo.mapTracker.contains(mapId)) { + deferredBufs = null; + return; + } + 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 chunk buffered in memory, then write the remaining + // of the block. + if (deferredBufs != null && !deferredBufs.isEmpty()) { + for (ByteBuffer deferredBuf : deferredBufs) { + writeBuf(deferredBuf); + } + deferredBufs = null; + } + 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 { + // 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 { + 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, + BlockPushException.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, + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + } + // Check if we can commit this block + if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { + // Identify duplicate block generated by speculative tasks. We respond success to + // the client in cases of duplicate even though no data is written. + if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || + partitionInfo.mapTracker.contains(mapId)) { + deferredBufs = null; + return; + } + if (partitionInfo.getCurrentMapId() < 0 && deferredBufs != null + && !deferredBufs.isEmpty()) { + for (ByteBuffer deferredBuf : deferredBufs) { + writeBuf(deferredBuf); + } + deferredBufs = null; + } + long updatedPos = partitionInfo.getPosition() + length; + if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { + partitionInfo.updateLastChunkOffset(updatedPos); + } + partitionInfo.setPosition(updatedPos); + partitionInfo.setCurrentMapId(-1); + + // update merged results + partitionInfo.blockMerged(mapId); + } else { + deferredBufs = null; + canWrite = false; + throw new RuntimeException(String.format("Couldn't find an opportunity to write " + + "block %s to merged shuffle", msg.blockId)); + } + } + } + + @Override + public void onFailure(String streamId, Throwable cause) throws IOException { + logger.error("Encountered issue when merging shuffle partition block {}", msg, cause); + // 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 (canWrite && partitionInfo != null && partitions.containsKey(partitionId)) { + synchronized (partitionInfo) { + 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 bitmaps = new LinkedList<>(); + List reduceIds = new LinkedList<>(); + List sizes = new LinkedList<>(); + Iterator> iterator = + partitions.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry 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.updateLastChunkOffset(partition.getPosition()); + } + 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.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 relativeAppPath) { + logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); + appsRelativePath.put(appId, Paths.get(relativeAppPath)); + } + + /** + * 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.data", appId, shuffleId, reduceId); + } + + String generateIndexFileName() { + return String.format("mergedShuffle_%s_%d_%d.index", 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 { + // The merged shuffle data file + final File targetFile; + 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 merged shuffle index file + private final FileChannel indexChannel; + private final DataOutputStream indexWriteStream; + // The offset for the last chunk tracked in the index file for this shuffle partition + private long lastChunkOffset; + + AppShufflePartitionInfo(File targetFile, File indexFile) throws IOException { + targetFile.createNewFile(); + this.targetFile = targetFile; + this.channel = new FileOutputStream(targetFile, true).getChannel(); + indexFile.createNewFile(); + FileOutputStream fos = new FileOutputStream(indexFile, true); + indexChannel = fos.getChannel(); + this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); + // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() + updateLastChunkOffset(0L); + this.position = 0; + this.encounteredFailure = false; + this.currentMapId = -1; + this.mapTracker = new RoaringBitmap(); + } + + public long getPosition() { + return position; + } + + public void setPosition(long position) { + this.position = position; + } + + boolean isEncounteredFailure() { + return encounteredFailure; + } + + void setEncounteredFailure(boolean encounteredFailure) { + this.encounteredFailure = encounteredFailure; + } + + int getCurrentMapId() { + return currentMapId; + } + + void setCurrentMapId(int mapId) { + this.currentMapId = mapId; + } + + long getLastChunkOffset() { + return lastChunkOffset; + } + + void blockMerged(int mapId) { + mapTracker.add(mapId); + } + + void updateLastChunkOffset(long lastChunkOffset) throws IOException { + long startPos = indexChannel.position(); + try { + indexWriteStream.writeLong(lastChunkOffset); + } catch(IOException ioe) { + // reset the position to avoid corrupting index files during exception. + indexChannel.position(startPos); + throw ioe; + } + this.lastChunkOffset = lastChunkOffset; + } + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java new file mode 100644 index 000000000000..a0d227bb8b1c --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -0,0 +1,152 @@ +/* + * 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 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.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; + +public class RemoteBlockPushResolverSuite { + + private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); + private static final String[] LOCAL_DIRS = new String[]{"target/l1", "target/l2"}; + private static final String TEST_APP = "testApp"; + + private TransportConf conf; + private RemoteBlockPushResolver pushResolver; + + @Before + public void before() throws IOException { + cleanupLocalDirs(); + for (String localDir : LOCAL_DIRS) { + Files.createDirectories(Paths.get(localDir).resolve(TEST_APP)); + } + MapConfigProvider provider = new MapConfigProvider( + ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); + conf = new TransportConf("shuffle", provider); + pushResolver = new RemoteBlockPushResolver(conf, LOCAL_DIRS); + pushResolver.registerApplication(TEST_APP, TEST_APP); + } + + @After + public void after() { + try { + pushResolver.applicationRemoved(TEST_APP, true); + cleanupLocalDirs(); + } catch (IOException e) { + // don't fail if clean up doesn't succeed. + log.warn("Error deleting test local dirs", e); + } + } + + private static void cleanupLocalDirs() throws IOException { + for (String local : LOCAL_DIRS) { + FileUtils.deleteDirectory(new File(local)); + } + } + + @Test(expected = RuntimeException.class) + public void testNoIndexFile() { + try { + pushResolver.getChunkCount(TEST_APP, 0, 0); + } catch (Throwable t) { + assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); + Throwables.propagate(t); + } + } + + @Test + public void testChunkCountsAndBlockData() 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(pushBlocks, blocks); + int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + assertEquals(2, numChunks); + validateChunks(0, 0, numChunks, new int[]{4, 5}); + } + + @Test + public void testMultipleBlocksInAChunk() 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(pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + assertEquals(3, numChunks); + validateChunks(0, 0, numChunks, new int[]{5, 5, 3}); + } + + private void validateChunks( + int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { + for (int i = 0; i < numChunks; i++) { + FileSegmentManagedBuffer mb = + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, + i); + assertEquals(expectedSizes[i], mb.getLength()); + } + } + + private void pushBlockHelper(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(TEST_APP, pushBlocks[i].blockId, 0)); + stream.onData(stream.getID(), blocks[i]); + stream.onComplete(stream.getID()); + } + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + } +} diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 3d14318bf90f..9f19b205ecb4 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -54,6 +55,7 @@ import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalBlockHandler; +import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.TransportConf; import org.apache.spark.network.yarn.util.HadoopConfigProvider; @@ -94,6 +96,12 @@ public class YarnShuffleService extends AuxiliaryService { 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. + private static final String YARN_LOCAL_DIRS = "yarn.nodemanager.local-dirs"; + private static final String MERGE_MANAGER_DIR = "merge_manager"; + protected static final String MERGE_DIR_RELATIVE_PATH = + "usercache/%s/appcache/%s/" + MERGE_MANAGER_DIR; + // just for testing when you want to find an open port @VisibleForTesting static int boundPort = -1; @@ -126,6 +134,7 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; + private RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -172,7 +181,10 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile); + String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)) + .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); + shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); + blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests @@ -276,6 +288,8 @@ public void initializeApplication(ApplicationInitializationContext context) { } secretManager.registerApp(appId, shuffleSecret); } + shuffleMergeManager.registerApplication( + appId, String.format(MERGE_DIR_RELATIVE_PATH, context.getUser(), appId)); } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } @@ -297,6 +311,9 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); + // TODO change cleanupLocalDirs to false. These should be deleted by yarn when the app + // finishes. + shuffleMergeManager.applicationRemoved(appId, true); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } From 64722673463bcb98143ab5bfd28205361b31e3be Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 12 May 2020 15:33:14 -0700 Subject: [PATCH 02/35] LIHADOOP-53438 Using different appId for the tests in RemoteBlockPushResolverSuite RB=2101153 BUG=LIHADOOP-53438 G=spark-reviewers R=mshen,yezhou A=yezhou --- .../shuffle/RemoteBlockPushResolver.java | 3 +- .../shuffle/RemoteBlockPushResolverSuite.java | 77 ++++++++++++------- 2 files changed, 51 insertions(+), 29 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 3b7d0d6148d6..86080fba9730 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -151,7 +151,8 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI File mergedShuffleFile = getMergedShuffleFile(id); if (!mergedShuffleFile.exists()) { throw new RuntimeException( - String.format("Application merged shuffle file is not found (id=%s)", id.toString())); + String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(), + id.toString())); } File indexFile = getMergedIndexFile(id); try { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index a0d227bb8b1c..0a9b2350ccfd 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -46,29 +46,25 @@ public class RemoteBlockPushResolverSuite { private static final Logger log = LoggerFactory.getLogger(RemoteBlockPushResolverSuite.class); - private static final String[] LOCAL_DIRS = new String[]{"target/l1", "target/l2"}; - private static final String TEST_APP = "testApp"; 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(); - for (String localDir : LOCAL_DIRS) { - Files.createDirectories(Paths.get(localDir).resolve(TEST_APP)); - } MapConfigProvider provider = new MapConfigProvider( ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); conf = new TransportConf("shuffle", provider); - pushResolver = new RemoteBlockPushResolver(conf, LOCAL_DIRS); - pushResolver.registerApplication(TEST_APP, TEST_APP); + pushResolver = new RemoteBlockPushResolver(conf, localDirs); } @After public void after() { try { - pushResolver.applicationRemoved(TEST_APP, true); cleanupLocalDirs(); } catch (IOException e) { // don't fail if clean up doesn't succeed. @@ -76,8 +72,8 @@ public void after() { } } - private static void cleanupLocalDirs() throws IOException { - for (String local : LOCAL_DIRS) { + private void cleanupLocalDirs() throws IOException { + for (String local : localDirs) { FileUtils.deleteDirectory(new File(local)); } } @@ -85,7 +81,10 @@ private static void cleanupLocalDirs() throws IOException { @Test(expected = RuntimeException.class) public void testNoIndexFile() { try { - pushResolver.getChunkCount(TEST_APP, 0, 0); + String appId = "app_NoIndexFile"; + registerApplication(appId); + pushResolver.getChunkCount(appId, 0, 0); + removeApplication(appId); } catch (Throwable t) { assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); Throwables.propagate(t); @@ -94,27 +93,32 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { + String appId = "app_ChunkCountsAndBlockData"; + registerApplication(appId); PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0), - new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), }; ByteBuffer[] blocks = new ByteBuffer[]{ ByteBuffer.wrap(new byte[4]), ByteBuffer.wrap(new byte[5]) }; - pushBlockHelper(pushBlocks, blocks); - int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + pushBlockHelper(appId, pushBlocks, blocks); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); assertEquals(2, numChunks); - validateChunks(0, 0, numChunks, new int[]{4, 5}); + validateChunks(appId,0, 0, numChunks, new int[]{4, 5}); + removeApplication(appId); } @Test public void testMultipleBlocksInAChunk() throws IOException { + String appId = "app_MultipleBlocksInAChunk"; + registerApplication(appId); 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), + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_2_0", 0), + new PushBlockStream(appId, "shuffle_0_3_0", 0), }; ByteBuffer[] buffers = new ByteBuffer[]{ ByteBuffer.wrap(new byte[2]), @@ -122,31 +126,48 @@ public void testMultipleBlocksInAChunk() throws IOException { ByteBuffer.wrap(new byte[5]), ByteBuffer.wrap(new byte[3]) }; - pushBlockHelper(pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(TEST_APP, 0, 0); + pushBlockHelper(appId, pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); assertEquals(3, numChunks); - validateChunks(0, 0, numChunks, new int[]{5, 5, 3}); + validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + removeApplication(appId); + } + + /** + * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. + * This is because when the application gets removed, the directory cleaner removes the merged + * data and file in a different thread which can delete the relevant data of a different test. + */ + private void registerApplication(String appId) throws IOException { + for (String localDir : localDirs) { + Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); + } + pushResolver.registerApplication(appId, appId + "/merge_manager"); + } + + private void removeApplication(String appId) { + pushResolver.applicationRemoved(appId, true); } private void validateChunks( - int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { + String appId, int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { for (int i = 0; i < numChunks; i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } - private void pushBlockHelper(PushBlockStream[] pushBlocks, ByteBuffer[] blocks) + 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(TEST_APP, pushBlocks[i].blockId, 0)); + new PushBlockStream(appId, pushBlocks[i].blockId, 0)); stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); } } From 1c78e1da1a57a8cc0e46682dffd64100163cbc10 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 14 May 2020 17:36:09 -0700 Subject: [PATCH 03/35] LIHADOOP-53496 Not logging all block push exceptions on the client RB=2104829 BUG=LIHADOOP-53496 G=spark-reviewers R=yezhou,mshen A=mshen --- .../apache/spark/network/shuffle/RemoteBlockPushResolver.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 86080fba9730..dd9d22f95403 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -472,8 +472,8 @@ public void onComplete(String streamId) throws IOException { } else { deferredBufs = null; canWrite = false; - throw new RuntimeException(String.format("Couldn't find an opportunity to write " - + "block %s to merged shuffle", msg.blockId)); + throw new RuntimeException(String.format("%s %s to merged shuffle", + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX, msg.blockId)); } } } From 71f32467257956bdf7171cf20fc8c92ed0a7e689 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Mon, 1 Jun 2020 14:23:33 -0700 Subject: [PATCH 04/35] LIHADOOP-53700 Separate configuration for caching the merged index files in NM RB=2130238 BUG=LIHADOOP-53700 G=spark-reviewers R=mshen,chsingh A=chsingh --- .../apache/spark/network/shuffle/RemoteBlockPushResolver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index dd9d22f95403..2ce6451f10d2 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -91,7 +91,7 @@ public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); - String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); + String indexCacheSize = conf.get("spark.shuffle.service.mergedIndex.cache.size", "100m"); CacheLoader indexCacheLoader = new CacheLoader() { public ShuffleIndexInformation load(File file) throws IOException { From 221178f59b9872ae55cad0f3974ae0c323563f13 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 10 Jun 2020 16:39:15 -0700 Subject: [PATCH 05/35] LIHADOOP-53940 Logging the data file and index file path when shuffle service is unable to create them RB=2146753 BUG=LIHADOOP-53940 G=spark-reviewers R=mshen,yezhou A=mshen,yezhou --- .../network/shuffle/RemoteBlockPushResolver.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 2ce6451f10d2..ebd29e7f0975 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -116,14 +116,18 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // 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); try { - File mergedShuffleFile = getMergedShuffleFile(key); if (mergedShuffleFile.exists()) { return null; } else { - return new AppShufflePartitionInfo(mergedShuffleFile, getMergedIndexFile(id)); + return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile); } } catch (IOException e) { + logger.error( + "Cannot create merged shuffle partition {} with shuffle file {} and index file {}", key, + mergedIndexFile.getAbsolutePath(), mergedIndexFile.getAbsolutePath()); throw new RuntimeException(String.format( "Cannot initialize merged shuffle partition %s", key.toString()), e); } @@ -196,7 +200,8 @@ private File getMergedIndexFile(AppShufflePartitionId id) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - Path relativeMergeDir = appsRelativePath.remove(appId); + Path relativeMergeDir = Preconditions.checkNotNull( + appsRelativePath.remove(appId), "application " + appId + " is not registered."); Iterator> iterator = partitions.entrySet().iterator(); while (iterator.hasNext()) { @@ -231,7 +236,7 @@ private void deleteExecutorDirs(Path[] dirs) { logger.debug("Successfully cleaned up directory: {}", localDir); } } catch (Exception e) { - logger.error("Failed to delete directory: " + localDir, e); + logger.error("Failed to delete directory: {}", localDir, e); } } } From 55b4a5fcf63c82ebbcd619402944470633d0ae11 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 15 Jun 2020 13:35:09 -0700 Subject: [PATCH 06/35] LIHADOOP-54059 LIHADOOP-53496 Handle the inconsistencies between local dirs provided to executor and the shuffle service and not log all exceptions at error/warning level RB=2152736 BUG=LIHADOOP-53496,LIHADOOP-54059 G=spark-reviewers R=yezhou,mshen A=mshen --- .../shuffle/RemoteBlockPushResolver.java | 74 +++++++++++++++---- .../shuffle/RemoteBlockPushResolverSuite.java | 35 +++++++-- 2 files changed, 91 insertions(+), 18 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index ebd29e7f0975..afc57e5a6f36 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -69,7 +69,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); private final Path[] localDirs; - private final ConcurrentMap appsRelativePath; + private final ConcurrentMap appsPathInfo; private final ConcurrentMap partitions; private final Executor directoryCleaner; @@ -86,7 +86,7 @@ public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { this.localDirs[i] = Paths.get(localDirs[i]); } this.partitions = Maps.newConcurrentMap(); - this.appsRelativePath = 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")); @@ -181,10 +181,13 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI // TODO should we use subDirsPerLocalDir to potentially reduce inode size? private File getFile(String appId, String filename) { int hash = JavaUtils.nonNegativeHash(filename); - Path localDir = localDirs[hash % localDirs.length]; - Path relativeMergeDir = Preconditions.checkNotNull( - appsRelativePath.get(appId), "application " + appId + " is not registered."); - return new File(localDir.resolve(relativeMergeDir).toFile(), 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 = appPathsInfo.getActiveLocalDirs(localDirs); + Path localDir = activeLocalDirs[hash % activeLocalDirs.length]; + return new File(localDir.resolve(appPathsInfo.relativeMergeDir).toFile(), filename); } private File getMergedShuffleFile(AppShufflePartitionId id) { @@ -200,8 +203,10 @@ private File getMergedIndexFile(AppShufflePartitionId id) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); - Path relativeMergeDir = Preconditions.checkNotNull( - appsRelativePath.remove(appId), "application " + appId + " is not registered."); + // 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."); Iterator> iterator = partitions.entrySet().iterator(); while (iterator.hasNext()) { @@ -219,8 +224,8 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(localDirs) - .map(dir -> dir.resolve(relativeMergeDir)).toArray(Path[]::new); + Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) + .map(dir -> dir.resolve(appPathsInfo.relativeMergeDir)).toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } @@ -484,8 +489,21 @@ public void onComplete(String streamId) throws IOException { } @Override - public void onFailure(String streamId, Throwable cause) throws IOException { - logger.error("Encountered issue when merging shuffle partition block {}", msg, cause); + public void onFailure(String streamId, Throwable throwable) throws IOException { + if ((throwable.getMessage() != null && + (throwable.getMessage().contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || + throwable.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) || + + (throwable.getCause() != null && throwable.getCause().getMessage() != null && + (throwable.getCause().getMessage().contains( + BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || + throwable.getCause().getMessage().contains( + BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { + logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); + } else { + logger.error("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 @@ -549,7 +567,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc @Override public void registerApplication(String appId, String relativeAppPath) { logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); - appsRelativePath.put(appId, Paths.get(relativeAppPath)); + appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); } /** @@ -688,4 +706,34 @@ void updateLastChunkOffset(long lastChunkOffset) throws IOException { this.lastChunkOffset = lastChunkOffset; } } + + /** + * Wraps all the information related to the merge_dir of an application. + */ + private static class AppPathsInfo { + + private final Path relativeMergeDir; + private Path[] activeLocalDirs; + + AppPathsInfo(Path relativeMergeDir) { + this.relativeMergeDir = Preconditions.checkNotNull( + relativeMergeDir, "relative merge directory path cannot be null"); + } + + private Path[] getActiveLocalDirs(Path[] localDirs) { + if (activeLocalDirs != null) { + return activeLocalDirs; + } + synchronized (this) { + activeLocalDirs = Arrays.stream(localDirs) + .filter(rootDir -> rootDir.resolve(relativeMergeDir).toFile().exists()) + .toArray(Path[]::new); + if (activeLocalDirs.length == 0) { + throw new RuntimeException( + "Did not find any active local directories wrt " + relativeMergeDir); + } + } + return activeLocalDirs; + } + } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 0a9b2350ccfd..fc8f7f8df7f7 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -22,6 +22,7 @@ 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; @@ -82,7 +83,7 @@ private void cleanupLocalDirs() throws IOException { public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; - registerApplication(appId); + registerApplication(appId, localDirs); pushResolver.getChunkCount(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -94,7 +95,7 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId); + registerApplication(appId, localDirs); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -113,7 +114,31 @@ public void testChunkCountsAndBlockData() throws IOException { @Test public void testMultipleBlocksInAChunk() throws IOException { String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId); + registerApplication(appId, localDirs); + PushBlockStream[] pushBlocks = new PushBlockStream[] { + new PushBlockStream(appId, "shuffle_0_0_0", 0), + new PushBlockStream(appId, "shuffle_0_1_0", 0), + new PushBlockStream(appId, "shuffle_0_2_0", 0), + new PushBlockStream(appId, "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(appId, pushBlocks, buffers); + int numChunks = pushResolver.getChunkCount(appId, 0, 0); + assertEquals(3, numChunks); + validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + removeApplication(appId); + } + + @Test + public void testAppUsingFewerLocalDirs() throws IOException { + String appId = "app_AppUsingFewerLocalDirs"; + String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); + registerApplication(appId, activeLocalDirs); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -138,8 +163,8 @@ public void testMultipleBlocksInAChunk() throws IOException { * This is because when the application gets removed, the directory cleaner removes the merged * data and file in a different thread which can delete the relevant data of a different test. */ - private void registerApplication(String appId) throws IOException { - for (String localDir : localDirs) { + private void registerApplication(String appId, String[] activeLocalDirs) throws IOException { + for (String localDir : activeLocalDirs) { Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); } pushResolver.registerApplication(appId, appId + "/merge_manager"); From f9d0e86d563643b434a368fcfae354a623b16b2f Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 24 Jun 2020 00:27:27 -0700 Subject: [PATCH 07/35] LIHADOOP-54379 Sorting the disks both on shuffle service and executors RB=2166324 BUG=LIHADOOP-54379 G=spark-reviewers R=yezhou,mshen A=mshen --- .../spark/network/shuffle/RemoteBlockPushResolver.java | 6 ++++++ .../org/apache/spark/network/yarn/YarnShuffleService.java | 5 +++-- 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index afc57e5a6f36..7f5e8aafcd42 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,6 +37,7 @@ 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; @@ -570,6 +571,11 @@ public void registerApplication(String appId, String relativeAppPath) { appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); } + @VisibleForTesting + public Path[] getLocalDirs() { + return 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. diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 9f19b205ecb4..0ed63576a75f 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -134,7 +134,8 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; - private RemoteBlockPushResolver shuffleMergeManager; + @VisibleForTesting + RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -181,7 +182,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)) + String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)).sorted() .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); From 548e2c03be7039bac41ba33c7159462730622c74 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 23 Jul 2020 17:43:50 -0700 Subject: [PATCH 08/35] LIHADOOP-52494 Magnet fallback to origin shuffle blocks when fetch of a shuffle chunk fails RB=2203642 BUG=LIHADOOP-52494 G=spark-reviewers R=yzhou,mshen,vsowrira A=mshen --- .../shuffle/RemoteBlockPushResolver.java | 108 +++++++++++++++--- .../shuffle/RemoteBlockPushResolverSuite.java | 33 +++--- 2 files changed, 112 insertions(+), 29 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 7f5e8aafcd42..df13546a1745 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -47,6 +47,8 @@ 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; @@ -54,6 +56,7 @@ 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; @@ -119,16 +122,18 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // 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(mergedShuffleFile, mergedIndexFile); + return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile, mergedMetaFile); } } catch (IOException e) { logger.error( - "Cannot create merged shuffle partition {} with shuffle file {} and index file {}", key, - mergedIndexFile.getAbsolutePath(), mergedIndexFile.getAbsolutePath()); + "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); } @@ -136,7 +141,10 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( } @Override - public int getChunkCount(String appId, int shuffleId, int reduceId) { + public MergedBlockMeta getMergedBlockMeta( + String appId, + int shuffleId, + int reduceId) { AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); File indexFile = getMergedIndexFile(id); if (!indexFile.exists()) { @@ -146,7 +154,18 @@ public int getChunkCount(String appId, int shuffleId, int reduceId) { } int size = (int) indexFile.length(); // First entry is the zero offset - return (size / Long.BYTES) - 1; + 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") @@ -201,6 +220,11 @@ private File getMergedIndexFile(AppShufflePartitionId id) { return getFile(id.appId, indexName); } + private File getMergedMetaFile(AppShufflePartitionId id) { + String metaName = id.generateMetaFileName(); + return getFile(id.appId, metaName); + } + @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { logger.info("Application {} removed, cleanupLocalDirs = {}", appId, cleanupLocalDirs); @@ -472,14 +496,19 @@ public void onComplete(String streamId) throws IOException { deferredBufs = null; } long updatedPos = partitionInfo.getPosition() + length; + boolean indexUpdated = false; if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { - partitionInfo.updateLastChunkOffset(updatedPos); + partitionInfo.updateLastChunkOffset(updatedPos, mapId); + indexUpdated = true; } partitionInfo.setPosition(updatedPos); partitionInfo.setCurrentMapId(-1); // update merged results partitionInfo.blockMerged(mapId); + if (indexUpdated) { + partitionInfo.resetChunkTracker(); + } } else { deferredBufs = null; canWrite = false; @@ -540,7 +569,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc try { partition.channel.truncate(partition.getPosition()); if (partition.getPosition() != partition.getLastChunkOffset()) { - partition.updateLastChunkOffset(partition.getPosition()); + partition.updateLastChunkOffset(partition.getPosition(), partition.lastMergedMapId); } bitmaps.add(partition.mapTracker); reduceIds.add(partitionId.reduceId); @@ -551,6 +580,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } 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, @@ -626,6 +656,10 @@ String generateIndexFileName() { return String.format("mergedShuffle_%s_%d_%d.index", appId, shuffleId, reduceId); } + String generateMetaFileName() { + return String.format("mergedShuffle_%s_%d_%d.meta", appId, shuffleId, reduceId); + } + boolean compareAppShuffleId(String appId, int shuffleId) { return Objects.equal(this.appId, appId) && this.shuffleId == shuffleId; } @@ -648,11 +682,17 @@ public static class AppShufflePartitionInfo { private RoaringBitmap mapTracker; // The merged shuffle index file private final FileChannel indexChannel; + 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; + + // Bitmap tracking which mapper's blocks are in shuffle chunk + private RoaringBitmap chunkTracker; + ByteBuf trackerBuf = null; - AppShufflePartitionInfo(File targetFile, File indexFile) throws IOException { + AppShufflePartitionInfo(File targetFile, File indexFile, File metaFile) throws IOException { targetFile.createNewFile(); this.targetFile = targetFile; this.channel = new FileOutputStream(targetFile, true).getChannel(); @@ -660,12 +700,15 @@ public static class AppShufflePartitionInfo { 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() - updateLastChunkOffset(0L); + updateLastChunkOffset(0L, -1); this.position = 0; this.encounteredFailure = false; - this.currentMapId = -1; this.mapTracker = new RoaringBitmap(); + this.chunkTracker = new RoaringBitmap(); } public long getPosition() { @@ -698,19 +741,54 @@ long getLastChunkOffset() { void blockMerged(int mapId) { mapTracker.add(mapId); + chunkTracker.add(mapId); + lastMergedMapId = mapId; } - void updateLastChunkOffset(long lastChunkOffset) throws IOException { - long startPos = indexChannel.position(); + void resetChunkTracker() { + chunkTracker.clear(); + } + + void updateLastChunkOffset(long lastChunkOffset, 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("{} updated index with offset {}", targetFile.getName(), lastChunkOffset); indexWriteStream.writeLong(lastChunkOffset); - } catch(IOException ioe) { - // reset the position to avoid corrupting index files during exception. - indexChannel.position(startPos); + } catch (IOException ioe) { + if (idxStartPos != -1) { + // reset the position to avoid corrupting index files during exception. + logger.warn("{} reset index to position {}", targetFile.getName(), idxStartPos); + indexChannel.position(idxStartPos); + } throw ioe; } this.lastChunkOffset = lastChunkOffset; } + + private void writeChunkTracker(int mapId) throws IOException { + if (mapId == -1) { + return; + } + chunkTracker.add(mapId); + if (trackerBuf == null) { + trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker)); + } + Encoders.Bitmaps.encode(trackerBuf, chunkTracker); + long metaStartPos = metaChannel.position(); + try { + logger.trace("{} write chunk to meta file", targetFile.getName()); + metaChannel.write(trackerBuf.nioBuffer()); + } catch (IOException ioe) { + logger.warn("{} reset position of meta file to {}", targetFile.getName(), metaStartPos); + metaChannel.position(metaStartPos); + throw ioe; + } finally { + trackerBuf.clear(); + } + } } /** diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index fc8f7f8df7f7..23bf7f4d970e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -32,6 +32,7 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.roaringbitmap.RoaringBitmap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,7 +85,7 @@ public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; registerApplication(appId, localDirs); - pushResolver.getChunkCount(appId, 0, 0); + pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { assertTrue(t.getMessage().startsWith("Application merged shuffle index file is not found")); @@ -105,9 +106,8 @@ public void testChunkCountsAndBlockData() throws IOException { ByteBuffer.wrap(new byte[5]) }; pushBlockHelper(appId, pushBlocks, blocks); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(2, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{4, 5}); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}}); removeApplication(appId); } @@ -128,9 +128,8 @@ public void testMultipleBlocksInAChunk() throws IOException { ByteBuffer.wrap(new byte[3]) }; pushBlockHelper(appId, pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(3, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + MergedBlockMeta meta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); removeApplication(appId); } @@ -152,9 +151,8 @@ public void testAppUsingFewerLocalDirs() throws IOException { ByteBuffer.wrap(new byte[3]) }; pushBlockHelper(appId, pushBlocks, buffers); - int numChunks = pushResolver.getChunkCount(appId, 0, 0); - assertEquals(3, numChunks); - validateChunks(appId,0, 0, numChunks, new int[]{5, 5, 3}); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); + validateChunks(appId, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); removeApplication(appId); } @@ -175,11 +173,18 @@ private void removeApplication(String appId) { } private void validateChunks( - String appId, int shuffleId, int reduceId, int numChunks, int[] expectedSizes) { - for (int i = 0; i < numChunks; i++) { + 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); + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } From 50efba9df9a91cad4708e41dad5a2cfba6993ae1 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 26 Aug 2020 16:28:24 -0700 Subject: [PATCH 09/35] LIHADOOP-55372 reduced the default for minChunkSizeInMergedShuffleFile RB=2253833 G=spark-reviewers R=mshen,vsowrira,mmuralid,yezhou A=mshen --- .../org/apache/spark/network/util/TransportConf.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 646e4278811f..cc79357fbbd6 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -363,4 +363,16 @@ public boolean useOldFetchProtocol() { return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false); } + /** + * The minimum size of a chunk when dividing a merged shuffle file in push based shuffle into + * multiple chunks. This is an optimization so that when push based shuffle merges multiple + * shuffle blocks belonging to the same shuffle partition into a merged shuffle file, it + * divides the merged shuffle file into multiple MB-sized chunks to optimize reading it later. + * 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"))); + } } From 8a6e01b1d37112a0427693f381307db8520a8617 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Tue, 8 Sep 2020 23:57:02 -0700 Subject: [PATCH 10/35] LIHADOOP-55315 Avoid network when fetching merged shuffle file in local host with a consistent view of app local dirs among different executors RB=2261073 BUG=LIHADOOP-55315 G=spark-reviewers R=chsingh,mshen,vsowrira,mmuralid A=mmuralid,chsingh --- .../shuffle/ExternalShuffleBlockResolver.java | 1 + .../shuffle/RemoteBlockPushResolver.java | 93 ++++++++++++------- .../shuffle/RemoteBlockPushResolverSuite.java | 36 +++++-- .../network/yarn/YarnShuffleService.java | 13 +-- 4 files changed, 90 insertions(+), 53 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index a095bf272341..f3dde738564e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -363,6 +363,7 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { public Map getLocalDirs(String appId, String[] execIds) { return Arrays.stream(execIds) + .filter(exec -> !exec.isEmpty()) .map(exec -> { ExecutorShuffleInfo info = executors.get(new AppExecId(appId, exec)); if (info == null) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index df13546a1745..e00928a2d2f5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,7 +37,6 @@ 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; @@ -71,24 +70,21 @@ 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 Path[] localDirs; private final ConcurrentMap appsPathInfo; private final ConcurrentMap partitions; private final Executor directoryCleaner; private final TransportConf conf; private final int minChunkSize; + private final String relativeMergeDirPathPattern; private final LoadingCache indexCache; @SuppressWarnings("UnstableApiUsage") - public RemoteBlockPushResolver(TransportConf conf, String[] localDirs) { + public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) { this.conf = conf; - this.localDirs = new Path[localDirs.length]; - for (int i = 0; i < localDirs.length; i++) { - this.localDirs[i] = Paths.get(localDirs[i]); - } this.partitions = Maps.newConcurrentMap(); this.appsPathInfo = Maps.newConcurrentMap(); this.directoryCleaner = Executors.newSingleThreadExecutor( @@ -106,6 +102,7 @@ public ShuffleIndexInformation load(File file) throws IOException { .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) .build(indexCacheLoader); + this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; } /** @@ -205,9 +202,22 @@ private File getFile(String appId, String filename) { AppPathsInfo appPathsInfo = Preconditions.checkNotNull( appsPathInfo.get(appId), "application " + appId + " is not registered or NM was restarted."); - Path[] activeLocalDirs = appPathsInfo.getActiveLocalDirs(localDirs); + Path[] activeLocalDirs = getActiveLocalDirs(appPathsInfo.activeLocalDirs); Path localDir = activeLocalDirs[hash % activeLocalDirs.length]; - return new File(localDir.resolve(appPathsInfo.relativeMergeDir).toFile(), filename); + String relativePath = getRelativePath(appPathsInfo.user, appId); + Path filePath = localDir.resolve(relativePath); + File targetFile = new File(filePath.toFile(), filename); + logger.info("Get the file for " + targetFile.getAbsolutePath()); + return targetFile; + } + + private Path[] getActiveLocalDirs(String[] activeLocalDirs) { + 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) { @@ -225,6 +235,16 @@ private File getMergedMetaFile(AppShufflePartitionId id) { 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."); + return Arrays.stream(appPathsInfo.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); @@ -249,8 +269,9 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) - .map(dir -> dir.resolve(appPathsInfo.relativeMergeDir)).toArray(Path[]::new); + Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs)) + .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) + .toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } @@ -596,14 +617,23 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } @Override - public void registerApplication(String appId, String relativeAppPath) { - logger.debug("register application with RemoteBlockPushResolver {} {}", appId, relativeAppPath); - appsPathInfo.put(appId, new AppPathsInfo(Paths.get(relativeAppPath))); + public void registerApplication(String appId, String user) { + logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user); + appsPathInfo.put(appId, new AppPathsInfo(user)); } - @VisibleForTesting - public Path[] getLocalDirs() { - return localDirs; + @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) -> + appPathsInfo.updateActiveLocalDirs( + targetAppId, relativeMergeDirPathPattern, localDirs)); } /** @@ -796,28 +826,23 @@ private void writeChunkTracker(int mapId) throws IOException { */ private static class AppPathsInfo { - private final Path relativeMergeDir; - private Path[] activeLocalDirs; + private final String user; + private String[] activeLocalDirs; - AppPathsInfo(Path relativeMergeDir) { - this.relativeMergeDir = Preconditions.checkNotNull( - relativeMergeDir, "relative merge directory path cannot be null"); + AppPathsInfo(String user) { + this.user = Preconditions.checkNotNull(user, "user cannot be null"); } - private Path[] getActiveLocalDirs(Path[] localDirs) { - if (activeLocalDirs != null) { - return activeLocalDirs; - } - synchronized (this) { + private AppPathsInfo updateActiveLocalDirs( + String appId, String relativePathPattern, String[] localDirs) { + if (activeLocalDirs == null) { + String relativePath = String.format(relativePathPattern, user, appId); activeLocalDirs = Arrays.stream(localDirs) - .filter(rootDir -> rootDir.resolve(relativeMergeDir).toFile().exists()) - .toArray(Path[]::new); - if (activeLocalDirs.length == 0) { - throw new RuntimeException( - "Did not find any active local directories wrt " + relativeMergeDir); - } + .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) + .toArray(String[]::new); + logger.info("Updated the active local dirs " + Arrays.toString(activeLocalDirs)); } - return activeLocalDirs; + return this; } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 23bf7f4d970e..b63a750ac298 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -52,6 +52,9 @@ public class RemoteBlockPushResolverSuite { private TransportConf conf; private RemoteBlockPushResolver pushResolver; private String[] localDirs; + private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/"; + private final String USER = "testuser"; + private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401"; @Before public void before() throws IOException { @@ -61,7 +64,7 @@ public void before() throws IOException { MapConfigProvider provider = new MapConfigProvider( ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); conf = new TransportConf("shuffle", provider); - pushResolver = new RemoteBlockPushResolver(conf, localDirs); + pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH); } @After @@ -84,7 +87,7 @@ private void cleanupLocalDirs() throws IOException { public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -96,7 +99,8 @@ public void testNoIndexFile() { @Test public void testChunkCountsAndBlockData() throws IOException { String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -114,7 +118,8 @@ public void testChunkCountsAndBlockData() throws IOException { @Test public void testMultipleBlocksInAChunk() throws IOException { String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId, localDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -137,7 +142,8 @@ public void testMultipleBlocksInAChunk() throws IOException { public void testAppUsingFewerLocalDirs() throws IOException { String appId = "app_AppUsingFewerLocalDirs"; String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerApplication(appId, activeLocalDirs); + registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { new PushBlockStream(appId, "shuffle_0_0_0", 0), new PushBlockStream(appId, "shuffle_0_1_0", 0), @@ -161,11 +167,23 @@ public void testAppUsingFewerLocalDirs() throws IOException { * This is because when the application gets removed, the directory cleaner removes the merged * data and file in a different thread which can delete the relevant data of a different test. */ - private void registerApplication(String appId, String[] activeLocalDirs) throws IOException { - for (String localDir : activeLocalDirs) { - Files.createDirectories(Paths.get(localDir).resolve(appId + "/merge_manager")); + 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)); } - pushResolver.registerApplication(appId, appId + "/merge_manager"); + } + + 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) { diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 0ed63576a75f..919458ac33d2 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.ByteBuffer; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -97,10 +96,7 @@ public class YarnShuffleService extends AuxiliaryService { private static final boolean DEFAULT_STOP_ON_FAILURE = false; // Used by shuffle merge manager to create merged shuffle files. - private static final String YARN_LOCAL_DIRS = "yarn.nodemanager.local-dirs"; - private static final String MERGE_MANAGER_DIR = "merge_manager"; - protected static final String MERGE_DIR_RELATIVE_PATH = - "usercache/%s/appcache/%s/" + MERGE_MANAGER_DIR; + protected static final String APP_BASE_RELATIVE_PATH = "usercache/%s/appcache/%s/"; // just for testing when you want to find an open port @VisibleForTesting @@ -182,9 +178,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - String[] localDirs = Arrays.stream(conf.getTrimmedStrings(YARN_LOCAL_DIRS)).sorted() - .map(dir -> new Path(dir).toUri().getPath()).toArray(String[]::new); - shuffleMergeManager = new RemoteBlockPushResolver(transportConf, localDirs); + shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a @@ -289,11 +283,10 @@ public void initializeApplication(ApplicationInitializationContext context) { } secretManager.registerApp(appId, shuffleSecret); } - shuffleMergeManager.registerApplication( - appId, String.format(MERGE_DIR_RELATIVE_PATH, context.getUser(), appId)); } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } + shuffleMergeManager.registerApplication(appId, context.getUser()); } @Override From ae5ffac10685a2e283953580bdd9793ba22116a3 Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Fri, 11 Sep 2020 17:03:30 -0700 Subject: [PATCH 11/35] LIHADOOP-55654 Duplicate application init calls trigger NPE and wrong local dirs update in shuffle service. Also fixing a memory leak. RB=2281730 BUG=LIHADOOP-55654 G=spark-reviewers R=vsowrira,chsingh,mshen A=vsowrira,chsingh --- .../shuffle/RemoteBlockPushResolver.java | 21 ++++++++---- .../shuffle/RemoteBlockPushResolverSuite.java | 32 +++++++++++++++++++ 2 files changed, 46 insertions(+), 7 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index e00928a2d2f5..cc316664c2fe 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -207,13 +207,15 @@ private File getFile(String appId, String filename) { String relativePath = getRelativePath(appPathsInfo.user, appId); Path filePath = localDir.resolve(relativePath); File targetFile = new File(filePath.toFile(), filename); - logger.info("Get the file for " + targetFile.getAbsolutePath()); + logger.debug("Get the file for {}", targetFile.getAbsolutePath()); return targetFile; } private Path[] getActiveLocalDirs(String[] activeLocalDirs) { - return Arrays.stream(activeLocalDirs) - .map(localDir -> Paths.get(localDir)).toArray(Path[]::new); + 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) { @@ -240,7 +242,11 @@ public String[] getMergedBlockDirs(String appId) { AppPathsInfo appPathsInfo = Preconditions.checkNotNull( appsPathInfo.get(appId), "application " + appId + " is not registered or NM was restarted."); - return Arrays.stream(appPathsInfo.activeLocalDirs) + 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); } @@ -250,7 +256,7 @@ 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.get(appId), + appsPathInfo.remove(appId), "application " + appId + " is not registered or NM was restarted."); Iterator> iterator = partitions.entrySet().iterator(); @@ -619,7 +625,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc @Override public void registerApplication(String appId, String user) { logger.debug("register application with RemoteBlockPushResolver {} {}", appId, user); - appsPathInfo.put(appId, new AppPathsInfo(user)); + appsPathInfo.putIfAbsent(appId, new AppPathsInfo(user)); } @Override @@ -840,7 +846,8 @@ private AppPathsInfo updateActiveLocalDirs( activeLocalDirs = Arrays.stream(localDirs) .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) .toArray(String[]::new); - logger.info("Updated the active local dirs " + Arrays.toString(activeLocalDirs)); + logger.info("Updated the active local dirs {} for application {}", + Arrays.toString(activeLocalDirs), appId); } return this; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index b63a750ac298..4f7a56cd41da 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -88,6 +88,7 @@ public void testNoIndexFile() { try { String appId = "app_NoIndexFile"; registerApplication(appId, USER); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); pushResolver.getMergedBlockMeta(appId, 0, 0); removeApplication(appId); } catch (Throwable t) { @@ -162,6 +163,37 @@ public void testAppUsingFewerLocalDirs() throws IOException { removeApplication(appId); } + @Test(expected = NullPointerException.class) + public void testUpdateLocalDirsOnlyOnce() throws IOException { + String appId = "app_App1"; + // First app init and executor register will store the active local dirs list + registerApplication(appId, USER); + String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + // Any later app init or executor register from the same application + // won't change the active local dirs list + registerApplication(appId, USER); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new); + registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( + "l2/usercache/testuser/appcache/app_App1/merge_manager")); + removeApplication(appId); + try { + pushResolver.getMergedBlockDirs(appId); + } catch (Throwable e) { + assertTrue(e.getMessage() + .startsWith("application app_App1 is not registered or NM was restarted.")); + Throwables.propagate(e); + } + } + /** * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. * This is because when the application gets removed, the directory cleaner removes the merged From e51042ba9a8e8589a6191f46e1e9e38ef755a7fa Mon Sep 17 00:00:00 2001 From: Min Shen Date: Wed, 23 Sep 2020 12:42:24 -0700 Subject: [PATCH 12/35] Further prune changes that should go into a later PR. --- .../spark/network/shuffle/ExternalShuffleBlockResolver.java | 1 - 1 file changed, 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index f3dde738564e..a095bf272341 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -363,7 +363,6 @@ public int removeBlocks(String appId, String execId, String[] blockIds) { public Map getLocalDirs(String appId, String[] execIds) { return Arrays.stream(execIds) - .filter(exec -> !exec.isEmpty()) .map(exec -> { ExecutorShuffleInfo info = executors.get(new AppExecId(appId, exec)); if (info == null) { From 83aca99746d4cacfee545d8c2ed2ff74fead3eb5 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 24 Jun 2020 00:27:27 -0700 Subject: [PATCH 13/35] LIHADOOP-54379 Sorting the disks both on shuffle service and executors RB=2166324 BUG=LIHADOOP-54379 G=spark-reviewers R=yezhou,mshen A=mshen --- .../spark/network/shuffle/RemoteBlockPushResolver.java | 6 ++++++ .../spark/network/yarn/YarnShuffleServiceSuite.scala | 10 ++++++++++ 2 files changed, 16 insertions(+) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index cc316664c2fe..ae591485cc6c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,6 +37,7 @@ 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; @@ -642,6 +643,11 @@ public void registerExecutor(String appId, String[] localDirs) { targetAppId, relativeMergeDirPathPattern, localDirs)); } + @VisibleForTesting + public Path[] getLocalDirs() { + return 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. diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index a6a302ad5df9..259c685a30ac 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -411,4 +411,14 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd )) } + test("NM local directories will be sorted") { + s1 = new YarnShuffleService + val localDir1 = "/tmp/b" + val localDir2 = "tmp/a" + yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir1 + "," + localDir2) + + s1.init(yarnConfig) + val expected = Array(localDir1, localDir2).sorted + s1.shuffleMergeManager.getLocalDirs.map(path => path.toString) should equal(expected) + } } From 04e0efe27325da1a07c530ab9342e9ef3fa1282a Mon Sep 17 00:00:00 2001 From: Ye Zhou Date: Tue, 11 Aug 2020 15:43:25 -0700 Subject: [PATCH 14/35] LIHADOOP-55022 Disable the merged shuffle file cleanup in stopApplication event in YarnShuffleService as NM is able to cleanup the app local dir in Hadoop 2.10 RB=2230078 BUG=LIHADOOP-55022 G=spark-reviewers R=chsingh,vsowrira A=chsingh,vsowrira,mshen --- .../org/apache/spark/network/yarn/YarnShuffleService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 919458ac33d2..2c3fe97bfddf 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -305,9 +305,9 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); - // TODO change cleanupLocalDirs to false. These should be deleted by yarn when the app - // finishes. - shuffleMergeManager.applicationRemoved(appId, true); + // Set cleanupLocalDirs to false as these merged shuffle files should be deleted + // by yarn when the app finishes in Hadoop 2.10 + shuffleMergeManager.applicationRemoved(appId, false); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } From 71dfd48d32b9ce797744a0fb2cb2db6d52e1e422 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 6 Oct 2020 13:53:15 -0700 Subject: [PATCH 15/35] Tests and cleanup --- .../spark/network/util/TransportConf.java | 22 +- .../shuffle/RemoteBlockPushResolver.java | 45 +- .../shuffle/RemoteBlockPushResolverSuite.java | 416 +++++++++++++++--- .../network/yarn/YarnShuffleService.java | 5 +- 4 files changed, 389 insertions(+), 99 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index cc79357fbbd6..0b9eb8457e4f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -364,15 +364,25 @@ public boolean useOldFetchProtocol() { } /** - * The minimum size of a chunk when dividing a merged shuffle file in push based shuffle into - * multiple chunks. This is an optimization so that when push based shuffle merges multiple - * shuffle blocks belonging to the same shuffle partition into a merged shuffle file, it - * divides the merged shuffle file into multiple MB-sized chunks to optimize reading it later. - * A corresponding index file for each merged shuffle file will be generated indicating chunk - * boundaries. + * 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. + */ + public long mergedIndexCacheSize() { + return JavaUtils.byteStringAsBytes( + conf.get("spark.shuffle.server.mergedIndexCacheSize", "100m")); + } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index ae591485cc6c..96e356ade4a7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -65,7 +65,7 @@ import org.apache.spark.network.util.TransportConf; /** - * An implementation of MergedShuffleFileManager that provides the most essential shuffle + * An implementation of {@link MergedShuffleFileManager} that provides the most essential shuffle * service processing logic to support push based shuffle. */ public class RemoteBlockPushResolver implements MergedShuffleFileManager { @@ -81,6 +81,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private final int minChunkSize; private final String relativeMergeDirPathPattern; + @SuppressWarnings("UnstableApiUsage") private final LoadingCache indexCache; @SuppressWarnings("UnstableApiUsage") @@ -92,7 +93,6 @@ public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPa // Add `spark` prefix because it will run in NM in Yarn mode. NettyUtils.createThreadFactory("spark-shuffle-merged-shuffle-directory-cleaner")); this.minChunkSize = conf.minChunkSizeInMergedShuffleFile(); - String indexCacheSize = conf.get("spark.shuffle.service.mergedIndex.cache.size", "100m"); CacheLoader indexCacheLoader = new CacheLoader() { public ShuffleIndexInformation load(File file) throws IOException { @@ -100,7 +100,7 @@ public ShuffleIndexInformation load(File file) throws IOException { } }; indexCache = CacheBuilder.newBuilder() - .maximumWeight(JavaUtils.byteStringAsBytes(indexCacheSize)) + .maximumWeight(conf.mergedIndexCacheSize()) .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) .build(indexCacheLoader); this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; @@ -208,7 +208,7 @@ private File getFile(String appId, String filename) { String relativePath = getRelativePath(appPathsInfo.user, appId); Path filePath = localDir.resolve(relativePath); File targetFile = new File(filePath.toFile(), filename); - logger.debug("Get the file for {}", targetFile.getAbsolutePath()); + logger.debug("Get merged file {}", targetFile.getAbsolutePath()); return targetFile; } @@ -526,7 +526,7 @@ public void onComplete(String streamId) throws IOException { long updatedPos = partitionInfo.getPosition() + length; boolean indexUpdated = false; if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { - partitionInfo.updateLastChunkOffset(updatedPos, mapId); + partitionInfo.updateChunkInfo(updatedPos, mapId); indexUpdated = true; } partitionInfo.setPosition(updatedPos); @@ -568,8 +568,10 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { // to write the block data to disk, we should also ignore here. if (canWrite && partitionInfo != null && partitions.containsKey(partitionId)) { synchronized (partitionInfo) { - partitionInfo.setCurrentMapId(-1); - partitionInfo.setEncounteredFailure(true); + if (partitionInfo.getCurrentMapId() >= 0 && partitionInfo.getCurrentMapId() == mapId) { + partitionInfo.setCurrentMapId(-1); + partitionInfo.setEncounteredFailure(true); + } } } } @@ -597,7 +599,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc try { partition.channel.truncate(partition.getPosition()); if (partition.getPosition() != partition.getLastChunkOffset()) { - partition.updateLastChunkOffset(partition.getPosition(), partition.lastMergedMapId); + partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId); } bitmaps.add(partition.mapTracker); reduceIds.add(partitionId.reduceId); @@ -722,15 +724,19 @@ public static class AppShufflePartitionInfo { private int currentMapId; // Bitmap tracking which mapper's blocks have been merged for this shuffle partition private RoaringBitmap mapTracker; - // The merged shuffle index file + // 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; + private int lastMergedMapId = -1; - // Bitmap tracking which mapper's blocks are in shuffle chunk + // Bitmap tracking which mapper's blocks are in the current shuffle chunk private RoaringBitmap chunkTracker; ByteBuf trackerBuf = null; @@ -746,7 +752,7 @@ public static class AppShufflePartitionInfo { metaChannel = new FileOutputStream(metaFile, true).getChannel(); this.currentMapId = -1; // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() - updateLastChunkOffset(0L, -1); + updateChunkInfo(0L, -1); this.position = 0; this.encounteredFailure = false; this.mapTracker = new RoaringBitmap(); @@ -791,14 +797,19 @@ void resetChunkTracker() { chunkTracker.clear(); } - void updateLastChunkOffset(long lastChunkOffset, int mapId) throws IOException { + /** + * 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("{} updated index with offset {}", targetFile.getName(), lastChunkOffset); - indexWriteStream.writeLong(lastChunkOffset); + logger.trace("{} updated index with offset {}", targetFile.getName(), chunkOffset); + indexWriteStream.writeLong(chunkOffset); } catch (IOException ioe) { if (idxStartPos != -1) { // reset the position to avoid corrupting index files during exception. @@ -807,7 +818,7 @@ void updateLastChunkOffset(long lastChunkOffset, int mapId) throws IOException { } throw ioe; } - this.lastChunkOffset = lastChunkOffset; + this.lastChunkOffset = chunkOffset; } private void writeChunkTracker(int mapId) throws IOException { @@ -834,7 +845,7 @@ private void writeChunkTracker(int mapId) throws IOException { } /** - * Wraps all the information related to the merge_dir of an application. + * Wraps all the information related to the merge directory of an application. */ private static class AppPathsInfo { diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 4f7a56cd41da..5ea0f447e9b8 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -45,16 +45,20 @@ 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; - private final String MERGE_DIR_RELATIVE_PATH = "usercache/%s/appcache/%s/"; - private final String USER = "testuser"; - private final String BLOCK_MANAGER_DIR = "blockmgr-193d8401"; @Before public void before() throws IOException { @@ -86,11 +90,10 @@ private void cleanupLocalDirs() throws IOException { @Test(expected = RuntimeException.class) public void testNoIndexFile() { try { - String appId = "app_NoIndexFile"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); - pushResolver.getMergedBlockMeta(appId, 0, 0); - removeApplication(appId); + 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); @@ -98,34 +101,32 @@ public void testNoIndexFile() { } @Test - public void testChunkCountsAndBlockData() throws IOException { - String appId = "app_ChunkCountsAndBlockData"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); + 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(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), + 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(appId, pushBlocks, blocks); - MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, blockMeta, new int[]{4, 5}, new int[][]{{0}, {1}}); - removeApplication(appId); + 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 testMultipleBlocksInAChunk() throws IOException { - String appId = "app_MultipleBlocksInAChunk"; - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, localDirs)); + 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(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), - new PushBlockStream(appId, "shuffle_0_2_0", 0), - new PushBlockStream(appId, "shuffle_0_3_0", 0), + 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]), @@ -133,23 +134,293 @@ public void testMultipleBlocksInAChunk() throws IOException { ByteBuffer.wrap(new byte[5]), ByteBuffer.wrap(new byte[3]) }; - pushBlockHelper(appId, pushBlocks, buffers); - MergedBlockMeta meta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, meta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); - removeApplication(appId); + 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])); + + // 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}}); + removeApplication(TEST_APP); + } + + @Test + public void testDeferredBufsAreWrittenDuringOnComplete() 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])); + 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}}); + removeApplication(TEST_APP); + } + + @Test + public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() 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])); + 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}}); + removeApplication(TEST_APP); + } + + @Test + public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() 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_0_0", 0); + StreamCallbackWithID stream2 = + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + // This should be ingored + 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}}); + removeApplication(TEST_APP); + } + + @Test + public void testFailureAfterData() throws IOException { + registerApplication(TEST_APP, TEST_USER); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); + 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()); + removeApplication(TEST_APP); + } + + @Test + public void testFailureAfterMultipleDataBlocks() throws IOException { + registerApplication(TEST_APP, TEST_USER); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); + 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()); + removeApplication(TEST_APP); + } + + @Test + public void testFailureAfterComplete() throws IOException { + registerApplication(TEST_APP, TEST_USER); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); + 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}}); + removeApplication(TEST_APP); + } + + @Test (expected = RuntimeException.class) + public void testTooLateArrival() 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]) + }; + 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}}); + removeApplication(TEST_APP); + throw re; + } + } + + @Test + public void testIncompleteStreamsAreOverwritten() 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[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}}); + removeApplication(TEST_APP); + } + + @Test (expected = RuntimeException.class) + public void testFailureWith3Streams() 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)); + // 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}}); + removeApplication(TEST_APP); + if (failedEx != null) { + throw failedEx; + } } @Test public void testAppUsingFewerLocalDirs() throws IOException { - String appId = "app_AppUsingFewerLocalDirs"; String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerApplication(appId, USER); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); + registerApplication(TEST_APP, TEST_USER); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); PushBlockStream[] pushBlocks = new PushBlockStream[] { - new PushBlockStream(appId, "shuffle_0_0_0", 0), - new PushBlockStream(appId, "shuffle_0_1_0", 0), - new PushBlockStream(appId, "shuffle_0_2_0", 0), - new PushBlockStream(appId, "shuffle_0_3_0", 0), + 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]), @@ -157,73 +428,72 @@ public void testAppUsingFewerLocalDirs() throws IOException { ByteBuffer.wrap(new byte[5]), ByteBuffer.wrap(new byte[3]) }; - pushBlockHelper(appId, pushBlocks, buffers); - MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(appId, 0, 0); - validateChunks(appId, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); - removeApplication(appId); + pushBlockHelper(TEST_APP, pushBlocks, buffers); + MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); + validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); + removeApplication(TEST_APP); } @Test(expected = NullPointerException.class) public void testUpdateLocalDirsOnlyOnce() throws IOException { - String appId = "app_App1"; // First app init and executor register will store the active local dirs list - registerApplication(appId, USER); + registerApplication(TEST_APP, TEST_USER); String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); // Any later app init or executor register from the same application // won't change the active local dirs list - registerApplication(appId, USER); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); + registerApplication(TEST_APP, TEST_USER); + assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new); - registerExecutor(appId, prepareBlockManagerLocalDirs(appId, USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(appId).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(appId)[0].contains( - "l2/usercache/testuser/appcache/app_App1/merge_manager")); - removeApplication(appId); + registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); + assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); + assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + removeApplication(TEST_APP); try { - pushResolver.getMergedBlockDirs(appId); + pushResolver.getMergedBlockDirs(TEST_APP); } catch (Throwable e) { assertTrue(e.getMessage() - .startsWith("application app_App1 is not registered or NM was restarted.")); + .startsWith("application " + TEST_APP + " is not registered or NM was restarted.")); Throwables.propagate(e); } } /** - * Registers the app with RemoteBlockPushResolver. Use a different appId for different tests. - * This is because when the application gets removed, the directory cleaner removes the merged - * data and file in a different thread which can delete the relevant data of a different test. + * 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)) { + private void registerExecutor(String TEST_APP, String[] localDirs) throws IOException { + pushResolver.registerExecutor(TEST_APP, localDirs); + for (String localDir : pushResolver.getMergedBlockDirs(TEST_APP)) { Files.createDirectories(Paths.get(localDir)); } } - private String[] prepareBlockManagerLocalDirs(String appId, String user, String[] localDirs){ + private String[] prepareBlockManagerLocalDirs(String TEST_APP, String user, String[] localDirs){ return Arrays.stream(localDirs) .map(localDir -> localDir + "/" + - String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, appId)) + String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, TEST_APP)) .toArray(String[]::new); } - private void removeApplication(String appId) { - pushResolver.applicationRemoved(appId, true); + private void removeApplication(String TEST_APP) { + // PushResolver cleans up the local dirs in a different thread which can conflict with the test data of other + // tests, since all the tests are using the same Application Id. + pushResolver.applicationRemoved(TEST_APP, false); } private void validateChunks( - String appId, int shuffleId, int reduceId, MergedBlockMeta meta, + String TEST_APP, int shuffleId, int reduceId, MergedBlockMeta meta, int[] expectedSizes, int[][] expectedMapsPerChunk) throws IOException { assertEquals("num chunks", expectedSizes.length, meta.getNumChunks()); RoaringBitmap[] bitmaps = meta.readChunkBitmaps(); @@ -234,20 +504,20 @@ private void validateChunks( } for (int i = 0; i < meta.getNumChunks(); i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } - private void pushBlockHelper(String appId, PushBlockStream[] pushBlocks, ByteBuffer[] blocks) + private void pushBlockHelper(String TEST_APP, 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)); + new PushBlockStream(TEST_APP, pushBlocks[i].blockId, 0)); stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); } } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 2c3fe97bfddf..548f7125aa68 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -179,7 +179,8 @@ protected void serviceInit(Configuration conf) throws Exception { TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); - blockHandler = new ExternalBlockHandler(transportConf, registeredExecutorFile, shuffleMergeManager); + blockHandler = new ExternalBlockHandler( + transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests @@ -305,8 +306,6 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); - // Set cleanupLocalDirs to false as these merged shuffle files should be deleted - // by yarn when the app finishes in Hadoop 2.10 shuffleMergeManager.applicationRemoved(appId, false); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); From 0c411c15c6f7c7ccd6e8a370884e8b912c1b939a Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 30 Sep 2020 20:22:39 -0700 Subject: [PATCH 16/35] LIHADOOP-55948 Failure in the push stream should not change the currentMap if it's not the active stream RB=2308973 BUG=LIHADOOP-55948 G=spark-reviewers R=mshen,vsowrira,yezhou,mmuralid A=mshen --- .../spark/network/util/TransportConf.java | 4 +- .../shuffle/RemoteBlockPushResolver.java | 162 +++++++++++------- .../yarn/YarnShuffleServiceSuite.scala | 10 -- 3 files changed, 105 insertions(+), 71 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 0b9eb8457e4f..1d17594a3d2e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -365,7 +365,7 @@ public boolean useOldFetchProtocol() { /** * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during - * push-based shuffle. + * 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 @@ -383,6 +383,6 @@ public int minChunkSizeInMergedShuffleFile() { */ public long mergedIndexCacheSize() { return JavaUtils.byteStringAsBytes( - conf.get("spark.shuffle.server.mergedIndexCacheSize", "100m")); + conf.get("spark.shuffle.server.mergedIndexCacheSize", "100m")); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 96e356ade4a7..e8395c0c9e39 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,7 +37,6 @@ 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; @@ -80,6 +79,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private final TransportConf conf; private final int minChunkSize; private final String relativeMergeDirPathPattern; + private final ErrorHandler.BlockPushErrorHandler errorHandler; @SuppressWarnings("UnstableApiUsage") private final LoadingCache indexCache; @@ -104,6 +104,7 @@ public ShuffleIndexInformation load(File file) throws IOException { .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) .build(indexCacheLoader); this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; + this.errorHandler = new ErrorHandler.BlockPushErrorHandler(); } /** @@ -125,7 +126,8 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( if (mergedShuffleFile.exists()) { return null; } else { - return new AppShufflePartitionInfo(mergedShuffleFile, mergedIndexFile, mergedMetaFile); + return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile, + mergedMetaFile); } } catch (IOException e) { logger.error( @@ -354,7 +356,11 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { return new StreamCallbackWithID() { private int length = 0; - private boolean canWrite = true; + // 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 the 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 deferredBufs; @@ -372,13 +378,61 @@ public String getID() { private void writeBuf(ByteBuffer buf) throws IOException { while (buf.hasRemaining()) { if (partitionInfo.isEncounteredFailure()) { - length += partitionInfo.channel.write(buf, partitionInfo.getPosition() + length); + 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 @@ -412,34 +466,23 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { return; } // Check whether we can write to disk - if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { - // Check 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. - if ((partitionInfo.getCurrentMapId() == mapId && length == 0) || - partitionInfo.mapTracker.contains(mapId)) { + 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 chunk buffered in memory, then write the remaining - // of the block. - if (deferredBufs != null && !deferredBufs.isEmpty()) { - for (ByteBuffer deferredBuf : deferredBufs) { - writeBuf(deferredBuf); - } - deferredBufs = null; - } + // 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 @@ -452,6 +495,8 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { 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 @@ -481,12 +526,14 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { @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, - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + 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. @@ -505,23 +552,19 @@ public void onComplete(String streamId) throws IOException { if (!partitions.containsKey(partitionId)) { deferredBufs = null; throw new RuntimeException(String.format("Block %s %s", msg.blockId, - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)); + ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); } // Check if we can commit this block - if (partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapId) { + 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 ((partitionInfo.getCurrentMapId() == mapId && length == 0) || - partitionInfo.mapTracker.contains(mapId)) { + if (isDuplicateBlock()) { deferredBufs = null; return; } - if (partitionInfo.getCurrentMapId() < 0 && deferredBufs != null - && !deferredBufs.isEmpty()) { - for (ByteBuffer deferredBuf : deferredBufs) { - writeBuf(deferredBuf); - } - deferredBufs = null; + if (partitionInfo.getCurrentMapId() < 0) { + writeAnyDeferredBlocks(); } long updatedPos = partitionInfo.getPosition() + length; boolean indexUpdated = false; @@ -539,25 +582,17 @@ public void onComplete(String streamId) throws IOException { } } else { deferredBufs = null; - canWrite = false; throw new RuntimeException(String.format("%s %s to merged shuffle", - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX, msg.blockId)); + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX, + msg.blockId)); } } + startBlockWrite = false; } @Override public void onFailure(String streamId, Throwable throwable) throws IOException { - if ((throwable.getMessage() != null && - (throwable.getMessage().contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || - throwable.getMessage().contains(BlockPushException.TOO_LATE_MESSAGE_SUFFIX))) || - - (throwable.getCause() != null && throwable.getCause().getMessage() != null && - (throwable.getCause().getMessage().contains( - BlockPushException.COULD_NOT_FIND_OPPORTUNITY_MSG_PREFIX) || - throwable.getCause().getMessage().contains( - BlockPushException.TOO_LATE_MESSAGE_SUFFIX)))) { + if (!errorHandler.shouldLogError(throwable)) { logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); } else { logger.error("Encountered issue when merging shuffle partition block {}", msg, throwable); @@ -566,12 +601,12 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { // 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 (canWrite && partitionInfo != null && partitions.containsKey(partitionId)) { + if (startBlockWrite && partitionInfo != null && partitions.containsKey(partitionId)) { synchronized (partitionInfo) { - if (partitionInfo.getCurrentMapId() >= 0 && partitionInfo.getCurrentMapId() == mapId) { - partitionInfo.setCurrentMapId(-1); - partitionInfo.setEncounteredFailure(true); - } + logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId, + partitionId.shuffleId, partitionId.reduceId); + partitionInfo.setCurrentMapId(-1); + partitionInfo.setEncounteredFailure(true); } } } @@ -645,11 +680,6 @@ public void registerExecutor(String appId, String[] localDirs) { targetAppId, relativeMergeDirPathPattern, localDirs)); } - @VisibleForTesting - public Path[] getLocalDirs() { - return 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. @@ -713,6 +743,8 @@ boolean compareAppShuffleId(String appId, int shuffleId) { * Metadata tracked for an actively merged shuffle partition */ public static class AppShufflePartitionInfo { + + private final AppShufflePartitionId partitionId; // The merged shuffle data file final File targetFile; public final FileChannel channel; @@ -740,7 +772,11 @@ public static class AppShufflePartitionInfo { private RoaringBitmap chunkTracker; ByteBuf trackerBuf = null; - AppShufflePartitionInfo(File targetFile, File indexFile, File metaFile) throws IOException { + AppShufflePartitionInfo(AppShufflePartitionId partitionId, + File targetFile, + File indexFile, + File metaFile) throws IOException { + this.partitionId = Preconditions.checkNotNull(partitionId, "partition id"); targetFile.createNewFile(); this.targetFile = targetFile; this.channel = new FileOutputStream(targetFile, true).getChannel(); @@ -764,6 +800,8 @@ public long getPosition() { } 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; } @@ -780,6 +818,8 @@ int getCurrentMapId() { } void setCurrentMapId(int mapId) { + logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}", + partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId); this.currentMapId = mapId; } @@ -788,6 +828,8 @@ long getLastChunkOffset() { } 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; @@ -808,7 +850,9 @@ void updateChunkInfo(long chunkOffset, int mapId) throws IOException { // update the chunk tracker to meta file before index file writeChunkTracker(mapId); idxStartPos = indexChannel.position(); - logger.trace("{} updated index with offset {}", targetFile.getName(), chunkOffset); + 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) { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 259c685a30ac..a6a302ad5df9 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -411,14 +411,4 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd )) } - test("NM local directories will be sorted") { - s1 = new YarnShuffleService - val localDir1 = "/tmp/b" - val localDir2 = "tmp/a" - yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir1 + "," + localDir2) - - s1.init(yarnConfig) - val expected = Array(localDir1, localDir2).sorted - s1.shuffleMergeManager.getLocalDirs.map(path => path.toString) should equal(expected) - } } From d0294638ea4df041057cc3549990e0c4ad9d57f0 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 15 Oct 2020 13:07:33 -0700 Subject: [PATCH 17/35] Minor style corrections --- .../shuffle/RemoteBlockPushResolver.java | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index e8395c0c9e39..026950913a6a 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -358,7 +358,7 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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 the partial written data. This is reset to false when the stream + // 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 @@ -592,10 +592,10 @@ public void onComplete(String streamId) throws IOException { @Override public void onFailure(String streamId, Throwable throwable) throws IOException { - if (!errorHandler.shouldLogError(throwable)) { - logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); - } else { + 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 @@ -758,10 +758,8 @@ public static class AppShufflePartitionInfo { 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. - */ + // 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 @@ -772,7 +770,8 @@ public static class AppShufflePartitionInfo { private RoaringBitmap chunkTracker; ByteBuf trackerBuf = null; - AppShufflePartitionInfo(AppShufflePartitionId partitionId, + AppShufflePartitionInfo( + AppShufflePartitionId partitionId, File targetFile, File indexFile, File metaFile) throws IOException { From 8f3839ffe4e5b2039f58260451041f5742ecafad Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 15 Oct 2020 13:56:34 -0700 Subject: [PATCH 18/35] Fixed style issues --- .../network/shuffle/RemoteBlockPushResolverSuite.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 5ea0f447e9b8..b325a486d109 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -487,8 +487,8 @@ private String[] prepareBlockManagerLocalDirs(String TEST_APP, String user, Stri } private void removeApplication(String TEST_APP) { - // PushResolver cleans up the local dirs in a different thread which can conflict with the test data of other - // tests, since all the tests are using the same Application Id. + // 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(TEST_APP, false); } @@ -504,7 +504,8 @@ private void validateChunks( } for (int i = 0; i < meta.getNumChunks(); i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, i); + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, + i); assertEquals(expectedSizes[i], mb.getLength()); } } From 1cd2d0365648ea9ff5201361f86ccc7f40bcdb12 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 19 Oct 2020 16:39:06 -0700 Subject: [PATCH 19/35] Renamed variables, methods, fixed indentation, addressed other review comments --- .../spark/network/util/TransportConf.java | 4 +- .../shuffle/RemoteBlockPushResolver.java | 304 ++++++++---------- .../shuffle/RemoteBlockPushResolverSuite.java | 142 ++++---- .../network/yarn/YarnShuffleService.java | 2 +- 4 files changed, 217 insertions(+), 235 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 1d17594a3d2e..81287438f7a9 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -375,7 +375,7 @@ public boolean useOldFetchProtocol() { */ public int minChunkSizeInMergedShuffleFile() { return Ints.checkedCast(JavaUtils.byteStringAsBytes( - conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m"))); + conf.get("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "2m"))); } /** @@ -383,6 +383,6 @@ public int minChunkSizeInMergedShuffleFile() { */ public long mergedIndexCacheSize() { return JavaUtils.byteStringAsBytes( - conf.get("spark.shuffle.server.mergedIndexCacheSize", "100m")); + conf.get("spark.shuffle.server.mergedIndexCacheSize", "100m")); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 026950913a6a..898f6933fce4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -90,81 +90,74 @@ public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPa 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")); + // 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 indexCacheLoader = - new CacheLoader() { - public ShuffleIndexInformation load(File file) throws IOException { - return new ShuffleIndexInformation(file); - } - }; + new CacheLoader() { + public ShuffleIndexInformation load(File file) throws IOException { + return new ShuffleIndexInformation(file); + } + }; indexCache = CacheBuilder.newBuilder() - .maximumWeight(conf.mergedIndexCacheSize()) - .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) - .build(indexCacheLoader); + .maximumWeight(conf.mergedIndexCacheSize()) + .weigher((Weigher) (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, + * 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) { + 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); + File dataFile = getMergedShuffleDataFile(id); + File indexFile = getMergedShuffleIndexFile(id); + File metaFile = getMergedShuffleMetaFile(id); try { - if (mergedShuffleFile.exists()) { + if (dataFile.exists()) { return null; } else { - return new AppShufflePartitionInfo(id, mergedShuffleFile, mergedIndexFile, - mergedMetaFile); + 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, mergedIndexFile.getAbsolutePath(), - mergedIndexFile.getAbsolutePath(), mergedMetaFile.getAbsolutePath()); - throw new RuntimeException(String.format( - "Cannot initialize merged shuffle partition %s", key.toString()), e); + "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) { + public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); - File indexFile = getMergedIndexFile(id); + File indexFile = getMergedShuffleIndexFile(id); if (!indexFile.exists()) { throw new RuntimeException( - String.format("Application merged shuffle index file is not found (id=%s)", - id.toString())); + 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); + File metaFile = getMergedShuffleMetaFile(id); if (!metaFile.exists()) { throw new RuntimeException( - String.format("Application merged shuffle meta file is not found (id=%s)", - id.toString())); + String.format("Application merged shuffle meta file is not found (id=%s)", id.toString())); } FileSegmentManagedBuffer chunkBitMaps = - new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length()); + new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length()); logger.trace( - "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks); + "{} shuffleId {} reduceId {} num chunks {}", appId, shuffleId, reduceId, numChunks); return new MergedBlockMeta(numChunks, chunkBitMaps); } @@ -172,23 +165,19 @@ public MergedBlockMeta getMergedBlockMeta( @Override public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceId, int chunkId) { AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); - File mergedShuffleFile = getMergedShuffleFile(id); + File mergedShuffleFile = getMergedShuffleDataFile(id); if (!mergedShuffleFile.exists()) { - throw new RuntimeException( - String.format("Merged shuffle file %s of %s not found", mergedShuffleFile.getPath(), - id.toString())); + throw new RuntimeException(String.format("Merged shuffle file %s of %s not found", + mergedShuffleFile.getPath(), id.toString())); } - File indexFile = getMergedIndexFile(id); + 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()); + conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); } catch (ExecutionException e) { throw new RuntimeException("Failed to open file: " + indexFile, e); } @@ -202,9 +191,8 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI 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."); + 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); @@ -216,53 +204,49 @@ private File getFile(String appId, String filename) { 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); + "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(); + private File getMergedShuffleDataFile(AppShufflePartitionId id) { + String fileName = String.format("%s.data", id.generateFileName()); return getFile(id.appId, fileName); } - private File getMergedIndexFile(AppShufflePartitionId id) { - String indexName = id.generateIndexFileName(); + private File getMergedShuffleIndexFile(AppShufflePartitionId id) { + String indexName = String.format("%s.index", id.generateFileName()); return getFile(id.appId, indexName); } - private File getMergedMetaFile(AppShufflePartitionId id) { - String metaName = id.generateMetaFileName(); + 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"); + 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); + .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."); + AppPathsInfo appPathsInfo = Preconditions.checkNotNull(appsPathInfo.remove(appId), + "application " + appId + " is not registered or NM was restarted."); Iterator> iterator = - partitions.entrySet().iterator(); + partitions.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); AppShufflePartitionId partitionId = entry.getKey(); @@ -279,8 +263,8 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { if (cleanupLocalDirs) { Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs)) - .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) - .toArray(Path[]::new); + .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) + .toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } @@ -308,11 +292,11 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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]); + AppShufflePartitionId partitionId = new AppShufflePartitionId( + msg.appId, Integer.parseInt(blockIdParts[1]), Integer.parseInt(blockIdParts[3])); + int mapIndex = Integer.parseInt(blockIdParts[2]); AppShufflePartitionInfo partitionInfoBeforeCheck = - getOrCreateAppShufflePartitionInfo(partitionId); + 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 @@ -352,7 +336,7 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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; + && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck; return new StreamCallbackWithID() { private int length = 0; @@ -360,7 +344,7 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { // 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; + private boolean isWriting = false; // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly private List deferredBufs; @@ -370,19 +354,20 @@ public String getID() { } /** - * 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. + * 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); + "{} 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); @@ -391,37 +376,37 @@ private void writeBuf(ByteBuffer buf) throws IOException { } /** - * 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. + * 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; + 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. + * 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); + 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. + * 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; @@ -467,17 +452,17 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { } // 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. - startBlockWrite = true; if (isDuplicateBlock()) { deferredBufs = null; return; } logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + partitionId.shuffleId, partitionId.reduceId); if (partitionInfo.getCurrentMapId() < 0) { - partitionInfo.setCurrentMapId(mapId); + partitionInfo.setCurrentMapId(mapIndex); } // If we got here, it's safe to write the block data to the merged shuffle file. We @@ -496,7 +481,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { } } else { logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + 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 @@ -527,7 +512,7 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { @Override public void onComplete(String streamId) throws IOException { logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + partitionId.shuffleId, partitionId.reduceId); if (partitionInfo == null) { if (isTooLate) { // Throw an exception here so the block data is drained from channel and server @@ -556,7 +541,7 @@ public void onComplete(String streamId) throws IOException { } // Check if we can commit this block if (allowedToWrite()) { - startBlockWrite = true; + 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()) { @@ -569,25 +554,25 @@ public void onComplete(String streamId) throws IOException { long updatedPos = partitionInfo.getPosition() + length; boolean indexUpdated = false; if (updatedPos - partitionInfo.getLastChunkOffset() >= minChunkSize) { - partitionInfo.updateChunkInfo(updatedPos, mapId); + partitionInfo.updateChunkInfo(updatedPos, mapIndex); indexUpdated = true; } partitionInfo.setPosition(updatedPos); partitionInfo.setCurrentMapId(-1); // update merged results - partitionInfo.blockMerged(mapId); + 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)); + ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX, + msg.blockId)); } } - startBlockWrite = false; + isWriting = false; } @Override @@ -601,10 +586,10 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { // 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)) { + if (isWriting && partitionInfo != null && partitions.containsKey(partitionId)) { synchronized (partitionInfo) { logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + partitionId.shuffleId, partitionId.reduceId); partitionInfo.setCurrentMapId(-1); partitionInfo.setEncounteredFailure(true); } @@ -613,6 +598,7 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { }; } + @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter") @Override public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId); @@ -620,7 +606,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc List reduceIds = new LinkedList<>(); List sizes = new LinkedList<>(); Iterator> iterator = - partitions.entrySet().iterator(); + partitions.entrySet().iterator(); while (iterator.hasNext()) { Map.Entry entry = iterator.next(); AppShufflePartitionId partitionId = entry.getKey(); @@ -649,7 +635,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc partition.indexWriteStream.close(); } catch (IOException closeEx) { logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId, - msg.shuffleId, partitionId.reduceId, closeEx); + msg.shuffleId, partitionId.reduceId, closeEx); } } } @@ -657,7 +643,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } 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)); + Ints.toArray(reduceIds), Longs.toArray(sizes)); } @Override @@ -669,20 +655,21 @@ public void registerApplication(String appId, String user) { @Override public void registerExecutor(String appId, String[] localDirs) { if (logger.isDebugEnabled()) { - logger.debug("register executor with RemoteBlockPushResolver {} {}", - appId, Arrays.toString(localDirs)); + 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) -> - appPathsInfo.updateActiveLocalDirs( - targetAppId, relativeMergeDirPathPattern, localDirs)); + "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. + * 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; @@ -705,7 +692,7 @@ public boolean equals(Object o) { } AppShufflePartitionId that = (AppShufflePartitionId) o; return shuffleId == that.shuffleId && reduceId == that.reduceId - && Objects.equal(appId, that.appId); + && Objects.equal(appId, that.appId); } @Override @@ -716,22 +703,14 @@ public int hashCode() { @Override public String toString() { return Objects.toStringHelper(this) - .add("appId", appId) - .add("shuffleId", shuffleId) - .add("reduceId", reduceId) - .toString(); + .add("appId", appId) + .add("shuffleId", shuffleId) + .add("reduceId", reduceId) + .toString(); } String generateFileName() { - return String.format("mergedShuffle_%s_%d_%d.data", appId, shuffleId, reduceId); - } - - String generateIndexFileName() { - return String.format("mergedShuffle_%s_%d_%d.index", appId, shuffleId, reduceId); - } - - String generateMetaFileName() { - return String.format("mergedShuffle_%s_%d_%d.meta", appId, shuffleId, reduceId); + return String.format("mergedShuffle_%s_%d_%d", appId, shuffleId, reduceId); } boolean compareAppShuffleId(String appId, int shuffleId) { @@ -739,14 +718,12 @@ boolean compareAppShuffleId(String appId, int shuffleId) { } } - /** - * Metadata tracked for an actively merged shuffle partition - */ + /** Metadata tracked for an actively merged shuffle partition */ public static class AppShufflePartitionInfo { private final AppShufflePartitionId partitionId; // The merged shuffle data file - final File targetFile; + final File dataFile; public final FileChannel channel; // Location offset of the last successfully merged block for this shuffle partition private long position; @@ -772,13 +749,13 @@ public static class AppShufflePartitionInfo { AppShufflePartitionInfo( AppShufflePartitionId partitionId, - File targetFile, + File dataFile, File indexFile, File metaFile) throws IOException { this.partitionId = Preconditions.checkNotNull(partitionId, "partition id"); - targetFile.createNewFile(); - this.targetFile = targetFile; - this.channel = new FileOutputStream(targetFile, true).getChannel(); + dataFile.createNewFile(); + this.dataFile = dataFile; + this.channel = new FileOutputStream(dataFile, true).getChannel(); indexFile.createNewFile(); FileOutputStream fos = new FileOutputStream(indexFile, true); indexChannel = fos.getChannel(); @@ -800,7 +777,7 @@ public long getPosition() { public void setPosition(long position) { logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId, this.position, position); + partitionId.shuffleId, partitionId.reduceId, this.position, position); this.position = position; } @@ -818,7 +795,7 @@ int getCurrentMapId() { void setCurrentMapId(int mapId) { logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}", - partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId); + partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId); this.currentMapId = mapId; } @@ -828,7 +805,7 @@ long getLastChunkOffset() { void blockMerged(int mapId) { logger.debug("{} shuffleId {} reduceId {} updated merging mapId {}", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId, mapId); + partitionId.shuffleId, partitionId.reduceId, mapId); mapTracker.add(mapId); chunkTracker.add(mapId); lastMergedMapId = mapId; @@ -840,8 +817,9 @@ void resetChunkTracker() { /** * 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. + * @param mapId the mapId to be added to chunk tracker. */ void updateChunkInfo(long chunkOffset, int mapId) throws IOException { long idxStartPos = -1; @@ -850,13 +828,13 @@ void updateChunkInfo(long chunkOffset, int mapId) throws IOException { writeChunkTracker(mapId); idxStartPos = indexChannel.position(); logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}", - partitionId.appId, partitionId.shuffleId, partitionId.reduceId, this.lastChunkOffset, - chunkOffset); + 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 {}", targetFile.getName(), idxStartPos); + logger.warn("{} reset index to position {}", dataFile.getName(), idxStartPos); indexChannel.position(idxStartPos); } throw ioe; @@ -875,10 +853,10 @@ private void writeChunkTracker(int mapId) throws IOException { Encoders.Bitmaps.encode(trackerBuf, chunkTracker); long metaStartPos = metaChannel.position(); try { - logger.trace("{} write chunk to meta file", targetFile.getName()); + logger.trace("{} write chunk to meta file", dataFile.getName()); metaChannel.write(trackerBuf.nioBuffer()); } catch (IOException ioe) { - logger.warn("{} reset position of meta file to {}", targetFile.getName(), metaStartPos); + logger.warn("{} reset position of meta file to {}", dataFile.getName(), metaStartPos); metaChannel.position(metaStartPos); throw ioe; } finally { @@ -904,10 +882,10 @@ private AppPathsInfo updateActiveLocalDirs( if (activeLocalDirs == null) { String relativePath = String.format(relativePathPattern, user, appId); activeLocalDirs = Arrays.stream(localDirs) - .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) - .toArray(String[]::new); + .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) + .toArray(String[]::new); logger.info("Updated the active local dirs {} for application {}", - Arrays.toString(activeLocalDirs), appId); + Arrays.toString(activeLocalDirs), appId); } return this; } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index b325a486d109..284db1365204 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -63,10 +63,10 @@ public class RemoteBlockPushResolverSuite { @Before public void before() throws IOException { localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(), - Paths.get("target/l2").toAbsolutePath().toString()}; + Paths.get("target/l2").toAbsolutePath().toString()}; cleanupLocalDirs(); MapConfigProvider provider = new MapConfigProvider( - ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); + ImmutableMap.of("spark.shuffle.server.minChunkSizeInMergedShuffleFile", "4")); conf = new TransportConf("shuffle", provider); pushResolver = new RemoteBlockPushResolver(conf, MERGE_DIR_RELATIVE_PATH); } @@ -105,12 +105,12 @@ 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), + 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]) + ByteBuffer.wrap(new byte[4]), + ByteBuffer.wrap(new byte[5]) }; pushBlockHelper(TEST_APP, pushBlocks, blocks); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); @@ -123,16 +123,16 @@ 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), + 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]) + 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); @@ -147,12 +147,12 @@ 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)); + 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)); + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); // This should be deferred stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[3])); @@ -177,12 +177,12 @@ 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)); + 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)); + 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])); @@ -207,14 +207,14 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOE PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); StreamCallbackWithID stream1 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0)); + 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)); + 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])); @@ -233,12 +233,12 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOE PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); StreamCallbackWithID stream1 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0)); + 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)); + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); // This should be ingored stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); @@ -263,7 +263,7 @@ 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)); + 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")); @@ -280,7 +280,7 @@ 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)); + 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])); @@ -299,7 +299,7 @@ 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)); + 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])); @@ -317,27 +317,27 @@ public void testTooLateArrival() 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_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]) + ByteBuffer.wrap(new byte[4]), + ByteBuffer.wrap(new byte[5]) }; StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( - new PushBlockStream(TEST_APP, pushBlocks[0].blockId, 0)); + 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)); + 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()); + "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}}); removeApplication(TEST_APP); @@ -351,14 +351,14 @@ 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)); + 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)); + 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)); @@ -374,18 +374,18 @@ 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)); + 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)); + 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)); + 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 @@ -394,8 +394,8 @@ public void testFailureWith3Streams() throws IOException { 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()); + "Couldn't find an opportunity to write block shuffle_0_2_0 to merged shuffle", + re.getMessage()); failedEx = re; } // stream 1 now completes @@ -417,16 +417,16 @@ public void testAppUsingFewerLocalDirs() throws IOException { registerApplication(TEST_APP, TEST_USER); registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); 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), + 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]) + 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 blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); @@ -442,24 +442,24 @@ public void testUpdateLocalDirsOnlyOnce() throws IOException { registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); // Any later app init or executor register from the same application // won't change the active local dirs list registerApplication(TEST_APP, TEST_USER); assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); activeLocalDirs = Arrays.stream(localDirs).toArray(String[]::new); registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); removeApplication(TEST_APP); try { pushResolver.getMergedBlockDirs(TEST_APP); } catch (Throwable e) { assertTrue(e.getMessage() - .startsWith("application " + TEST_APP + " is not registered or NM was restarted.")); + .startsWith("application " + TEST_APP + " is not registered or NM was restarted.")); Throwables.propagate(e); } } @@ -471,30 +471,33 @@ private void registerApplication(String appId, String user) throws IOException { pushResolver.registerApplication(appId, user); } - private void registerExecutor(String TEST_APP, String[] localDirs) throws IOException { - pushResolver.registerExecutor(TEST_APP, localDirs); - for (String localDir : pushResolver.getMergedBlockDirs(TEST_APP)) { + 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 TEST_APP, String user, String[] localDirs){ + 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, TEST_APP)) - .toArray(String[]::new); + .map(localDir -> + localDir + "/" + String.format(MERGE_DIR_RELATIVE_PATH + BLOCK_MANAGER_DIR, user, appId)) + .toArray(String[]::new); } - private void removeApplication(String TEST_APP) { + 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(TEST_APP, false); + pushResolver.applicationRemoved(appId, false); } private void validateChunks( - String TEST_APP, int shuffleId, int reduceId, MergedBlockMeta meta, - int[] expectedSizes, int[][] expectedMapsPerChunk) throws IOException { + 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); @@ -504,21 +507,22 @@ private void validateChunks( } for (int i = 0; i < meta.getNumChunks(); i++) { FileSegmentManagedBuffer mb = - (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(TEST_APP, shuffleId, reduceId, - i); + (FileSegmentManagedBuffer) pushResolver.getMergedBlockData(appId, shuffleId, reduceId, i); assertEquals(expectedSizes[i], mb.getLength()); } } - private void pushBlockHelper(String TEST_APP, PushBlockStream[] pushBlocks, ByteBuffer[] blocks) - throws IOException { + 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(TEST_APP, pushBlocks[i].blockId, 0)); + new PushBlockStream(appId, pushBlocks[i].blockId, 0)); stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); + pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); } } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 548f7125aa68..b1fe64077558 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -180,7 +180,7 @@ protected void serviceInit(Configuration conf) throws Exception { TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); blockHandler = new ExternalBlockHandler( - transportConf, registeredExecutorFile, shuffleMergeManager); + transportConf, registeredExecutorFile, shuffleMergeManager); // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests From 3356c19f9013b8df4274205fc4d912196e7750be Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 22 Oct 2020 17:11:02 -0700 Subject: [PATCH 20/35] Addressing review comments --- .../shuffle/RemoteBlockPushResolver.java | 120 +++++++++--------- .../shuffle/RemoteBlockPushResolverSuite.java | 114 ++++------------- 2 files changed, 87 insertions(+), 147 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 898f6933fce4..67709b9d10dc 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -143,16 +143,16 @@ public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduc 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())); + 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("Application merged shuffle meta file is not found (id=%s)", id.toString())); + 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()); @@ -165,10 +165,10 @@ public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduc @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 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 { @@ -177,9 +177,11 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI ShuffleIndexInformation shuffleIndexInformation = indexCache.get(indexFile); ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(chunkId); return new FileSegmentManagedBuffer( - conf, mergedShuffleFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); + conf, dataFile, shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); } catch (ExecutionException e) { - throw new RuntimeException("Failed to open file: " + indexFile, e); + throw new RuntimeException(String.format( + "Failed to open merged shuffle index file %s of %s", indexFile.getPath(), id.toString()), + e); } } @@ -334,7 +336,7 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { // 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 + // 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; @@ -384,23 +386,24 @@ private void writeBuf(ByteBuffer buf) throws IOException { */ private boolean allowedToWrite() { assert partitionInfo != null; - return partitionInfo.getCurrentMapId() < 0 || partitionInfo.getCurrentMapId() == mapIndex; + 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 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. + * 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.getCurrentMapId() == mapIndex && length == 0) + return (partitionInfo.getCurrentMapIndex() == mapIndex && length == 0) || partitionInfo.mapTracker.contains(mapIndex); } @@ -408,7 +411,7 @@ private boolean isDuplicateBlock() { * 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 { + private void writeAnyDeferredBufs() throws IOException { assert partitionInfo != null; if (deferredBufs != null && !deferredBufs.isEmpty()) { for (ByteBuffer deferredBuf : deferredBufs) { @@ -433,8 +436,8 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // 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 + // 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 @@ -461,13 +464,13 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { } logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId, partitionId.shuffleId, partitionId.reduceId); - if (partitionInfo.getCurrentMapId() < 0) { - partitionInfo.setCurrentMapId(mapIndex); + 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. - writeAnyDeferredBlocks(); + 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 @@ -500,7 +503,9 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { if (deferredBufs == null) { deferredBufs = new LinkedList<>(); } - // Write the buffer to the in-memory deferred cache + // 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(); @@ -548,8 +553,8 @@ public void onComplete(String streamId) throws IOException { deferredBufs = null; return; } - if (partitionInfo.getCurrentMapId() < 0) { - writeAnyDeferredBlocks(); + if (partitionInfo.getCurrentMapIndex() < 0) { + writeAnyDeferredBufs(); } long updatedPos = partitionInfo.getPosition() + length; boolean indexUpdated = false; @@ -558,7 +563,7 @@ public void onComplete(String streamId) throws IOException { indexUpdated = true; } partitionInfo.setPosition(updatedPos); - partitionInfo.setCurrentMapId(-1); + partitionInfo.setCurrentMapIndex(-1); // update merged results partitionInfo.blockMerged(mapIndex); @@ -590,7 +595,7 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { synchronized (partitionInfo) { logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId, partitionId.shuffleId, partitionId.reduceId); - partitionInfo.setCurrentMapId(-1); + partitionInfo.setCurrentMapIndex(-1); partitionInfo.setEncounteredFailure(true); } } @@ -620,7 +625,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc try { partition.channel.truncate(partition.getPosition()); if (partition.getPosition() != partition.getLastChunkOffset()) { - partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapId); + partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex); } bitmaps.add(partition.mapTracker); reduceIds.add(partitionId.reduceId); @@ -729,19 +734,19 @@ public static class AppShufflePartitionInfo { 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; + // 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 ids that belong to every - // chunk. The entry per chunk is a serialized bitmap. + // 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 lastMergedMapId = -1; + private int lastMergedMapIndex = -1; // Bitmap tracking which mapper's blocks are in the current shuffle chunk private RoaringBitmap chunkTracker; @@ -762,7 +767,7 @@ public static class AppShufflePartitionInfo { this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); metaFile.createNewFile(); metaChannel = new FileOutputStream(metaFile, true).getChannel(); - this.currentMapId = -1; + this.currentMapIndex = -1; // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() updateChunkInfo(0L, -1); this.position = 0; @@ -789,26 +794,26 @@ void setEncounteredFailure(boolean encounteredFailure) { this.encounteredFailure = encounteredFailure; } - int getCurrentMapId() { - return currentMapId; + int getCurrentMapIndex() { + return currentMapIndex; } - void setCurrentMapId(int mapId) { - logger.trace("{} shuffleId {} reduceId {} updated mapId {} current mapId {}", - partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapId, mapId); - this.currentMapId = mapId; + 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 mapId) { - logger.debug("{} shuffleId {} reduceId {} updated merging mapId {}", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId, mapId); - mapTracker.add(mapId); - chunkTracker.add(mapId); - lastMergedMapId = mapId; + 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() { @@ -816,16 +821,17 @@ void resetChunkTracker() { } /** - * Appends the chunk offset to the index file and adds the mapId to the chunk tracker. + * 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. * * @param chunkOffset the offset of the chunk in the data file. - * @param mapId the mapId to be added to chunk tracker. + * @param mapIndex the map index to be added to chunk tracker. */ - void updateChunkInfo(long chunkOffset, int mapId) throws IOException { + void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException { long idxStartPos = -1; try { // update the chunk tracker to meta file before index file - writeChunkTracker(mapId); + writeChunkTracker(mapIndex); idxStartPos = indexChannel.position(); logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}", partitionId.appId, partitionId.shuffleId, partitionId.reduceId, this.lastChunkOffset, @@ -842,11 +848,11 @@ void updateChunkInfo(long chunkOffset, int mapId) throws IOException { this.lastChunkOffset = chunkOffset; } - private void writeChunkTracker(int mapId) throws IOException { - if (mapId == -1) { + private void writeChunkTracker(int mapIndex) throws IOException { + if (mapIndex == -1) { return; } - chunkTracker.add(mapId); + chunkTracker.add(mapIndex); if (trackerBuf == null) { trackerBuf = Unpooled.buffer(Encoders.Bitmaps.encodedLength(chunkTracker)); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 284db1365204..f74648f6149e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -69,15 +69,18 @@ public void before() throws IOException { 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(); - } catch (IOException e) { + removeApplication(TEST_APP); + } catch (Exception e) { // don't fail if clean up doesn't succeed. - log.warn("Error deleting test local dirs", e); + log.debug("Error while tearing down", e); } } @@ -90,20 +93,15 @@ private void cleanupLocalDirs() throws IOException { @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")); + assertTrue(t.getMessage().startsWith("Merged shuffle index file")); 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), @@ -115,13 +113,10 @@ public void testBasicBlockMerge() throws IOException { 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), @@ -137,14 +132,10 @@ public void testDividingMergedBlocksIntoChunks() throws IOException { 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)); @@ -167,14 +158,10 @@ public void testDeferredBufsAreWrittenDuringOnData() throws IOException { 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}}); - removeApplication(TEST_APP); } @Test public void testDeferredBufsAreWrittenDuringOnComplete() 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)); @@ -197,14 +184,10 @@ public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException { 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}}); - removeApplication(TEST_APP); } @Test public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() 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)); @@ -223,14 +206,10 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOE 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}}); - removeApplication(TEST_APP); } @Test public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() 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)); @@ -239,7 +218,7 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOE 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 ingored + // This should be ignored stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); stream2.onData(stream2.getID(), ByteBuffer.wrap(new byte[2])); @@ -253,15 +232,11 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOE 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}}); - removeApplication(TEST_APP); } @Test public void testFailureAfterData() throws IOException { - registerApplication(TEST_APP, TEST_USER); - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); 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])); @@ -270,15 +245,11 @@ public void testFailureAfterData() throws IOException { pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); assertEquals("num-chunks", 0, blockMeta.getNumChunks()); - removeApplication(TEST_APP); } @Test public void testFailureAfterMultipleDataBlocks() throws IOException { - registerApplication(TEST_APP, TEST_USER); - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); 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])); @@ -289,15 +260,11 @@ public void testFailureAfterMultipleDataBlocks() throws IOException { pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(TEST_APP, 0)); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); assertEquals("num-chunks", 0, blockMeta.getNumChunks()); - removeApplication(TEST_APP); } @Test public void testFailureAfterComplete() throws IOException { - registerApplication(TEST_APP, TEST_USER); - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); 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])); @@ -309,13 +276,10 @@ public void testFailureAfterComplete() throws IOException { 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}}); - removeApplication(TEST_APP); } @Test (expected = RuntimeException.class) public void testTooLateArrival() 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)}; @@ -340,14 +304,12 @@ public void testTooLateArrival() throws IOException { "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}}); - removeApplication(TEST_APP); throw re; } } @Test public void testIncompleteStreamsAreOverwritten() 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 = @@ -364,14 +326,10 @@ public void testIncompleteStreamsAreOverwritten() throws IOException { 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}}); - removeApplication(TEST_APP); } @Test (expected = RuntimeException.class) public void testFailureWith3Streams() 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)); @@ -405,61 +363,37 @@ public void testFailureWith3Streams() throws IOException { 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}}); - removeApplication(TEST_APP); if (failedEx != null) { throw failedEx; } } - @Test - public void testAppUsingFewerLocalDirs() throws IOException { - String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerApplication(TEST_APP, TEST_USER); - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); - 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 blockMeta = pushResolver.getMergedBlockMeta(TEST_APP, 0, 0); - validateChunks(TEST_APP, 0, 0, blockMeta, new int[]{5, 5, 3}, new int[][]{{0, 1}, {2}, {3}}); - removeApplication(TEST_APP); - } - @Test(expected = NullPointerException.class) public void testUpdateLocalDirsOnlyOnce() throws IOException { - // First app init and executor register will store the active local dirs list - registerApplication(TEST_APP, TEST_USER); + String testApp = "updateLocalDirsOnlyOnceTest"; + registerApplication(testApp, TEST_USER); String[] activeLocalDirs = Arrays.stream(localDirs).skip(1).toArray(String[]::new); - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + 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(TEST_APP, TEST_USER); - assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); + 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(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, activeLocalDirs)); - assertEquals(pushResolver.getMergedBlockDirs(TEST_APP).length, 1); - assertTrue(pushResolver.getMergedBlockDirs(TEST_APP)[0].contains( - "l2/usercache/" + TEST_USER + "/appcache/" + TEST_APP + "/merge_manager")); - removeApplication(TEST_APP); + 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(TEST_APP); + pushResolver.getMergedBlockDirs(testApp); } catch (Throwable e) { assertTrue(e.getMessage() - .startsWith("application " + TEST_APP + " is not registered or NM was restarted.")); + .startsWith("application " + testApp + " is not registered or NM was restarted.")); Throwables.propagate(e); } } From d879beb5ab9047e05484d181ae953570cf38c2b3 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Sun, 25 Oct 2020 17:17:13 -0700 Subject: [PATCH 21/35] Changed the partitions map and addressed other review comments --- .../spark/network/util/TransportConf.java | 2 +- .../shuffle/RemoteBlockPushResolver.java | 244 +++++++++--------- 2 files changed, 126 insertions(+), 120 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 81287438f7a9..196eb776d61b 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -379,7 +379,7 @@ public int minChunkSizeInMergedShuffleFile() { } /** - * The size of cache used in push-based shuffle for storing merged index files. + * The size of cache in memory which is used in push-based shuffle for storing merged index files. */ public long mergedIndexCacheSize() { return JavaUtils.byteStringAsBytes( diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 67709b9d10dc..1a7481cf422c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -73,7 +73,7 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private static final String MERGE_MANAGER_DIR = "merge_manager"; private final ConcurrentMap appsPathInfo; - private final ConcurrentMap partitions; + private final ConcurrentMap> partitions; private final Executor directoryCleaner; private final TransportConf conf; @@ -108,24 +108,28 @@ public ShuffleIndexInformation load(File file) throws IOException { } /** - * 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. + * 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(AppShufflePartitionId id) { - return partitions.computeIfAbsent(id, key -> { + private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( + AppShuffleId appShuffleId, + int reduceId) { + Map 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(id); - File indexFile = getMergedShuffleIndexFile(id); - File metaFile = getMergedShuffleMetaFile(id); + 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(id, dataFile, indexFile, metaFile); + return new AppShufflePartitionInfo(appShuffleId, reduceId, dataFile, indexFile, metaFile); } } catch (IOException e) { logger.error( @@ -140,19 +144,19 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo(AppShufflePar @Override public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduceId) { - AppShufflePartitionId id = new AppShufflePartitionId(appId, shuffleId, reduceId); - File indexFile = getMergedShuffleIndexFile(id); + AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId); + File indexFile = getMergedShuffleIndexFile(appShuffleId, reduceId); if (!indexFile.exists()) { throw new RuntimeException(String.format( - "Merged shuffle index file %s of %s not found", indexFile.getPath(), id.toString())); + "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(id); + File metaFile = getMergedShuffleMetaFile(appShuffleId, reduceId); if (!metaFile.exists()) { - throw new RuntimeException(String.format("Merged shuffle meta file %s of %s not found", - metaFile.getPath(), id.toString())); + throw new RuntimeException(String.format("Merged shuffle meta file %s not found", + metaFile.getPath())); } FileSegmentManagedBuffer chunkBitMaps = new FileSegmentManagedBuffer(conf, metaFile, 0L, metaFile.length()); @@ -164,13 +168,13 @@ public MergedBlockMeta getMergedBlockMeta(String appId, int shuffleId, int reduc @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); + AppShuffleId appShuffleId = new AppShuffleId(appId, shuffleId); + File dataFile = getMergedShuffleDataFile(appShuffleId, reduceId); if (!dataFile.exists()) { - throw new RuntimeException(String.format("Merged shuffle data file %s of %s not found", - dataFile.getPath(), id.toString())); + throw new RuntimeException(String.format("Merged shuffle data file %s not found", + dataFile.getPath())); } - File indexFile = getMergedShuffleIndexFile(id); + 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. @@ -180,8 +184,7 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI 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); + "Failed to open merged shuffle index file %s", indexFile.getPath()), e); } } @@ -214,19 +217,19 @@ 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 getMergedShuffleDataFile(AppShuffleId appShuffleId, int reduceId) { + String fileName = String.format("%s.data", generateFileName(appShuffleId, reduceId)); + return getFile(appShuffleId.appId, fileName); } - private File getMergedShuffleIndexFile(AppShufflePartitionId id) { - String indexName = String.format("%s.index", id.generateFileName()); - return getFile(id.appId, indexName); + private File getMergedShuffleIndexFile(AppShuffleId appShuffleId, int reduceId) { + String indexName = String.format("%s.index", generateFileName(appShuffleId, reduceId)); + return getFile(appShuffleId.appId, indexName); } - private File getMergedShuffleMetaFile(AppShufflePartitionId id) { - String metaName = String.format("%s.meta", id.generateFileName()); - return getFile(id.appId, metaName); + private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) { + String metaName = String.format("%s.meta", generateFileName(appShuffleId, reduceId)); + return getFile(appShuffleId.appId, metaName); } @Override @@ -247,22 +250,23 @@ public void applicationRemoved(String appId, boolean 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> iterator = + Iterator>> iterator = partitions.entrySet().iterator(); while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - AppShufflePartitionId partitionId = entry.getKey(); - AppShufflePartitionInfo partition = entry.getValue(); - if (appId.equals(partitionId.appId)) { + Map.Entry> entry = iterator.next(); + AppShuffleId appShuffleId = entry.getKey(); + if (appId.equals(appShuffleId.appId)) { iterator.remove(); - try { - partition.channel.close(); - } catch (IOException e) { - logger.error("Error closing merged shuffle file for {}", partitionId); + 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))) @@ -294,11 +298,11 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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])); + 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(partitionId); + 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 @@ -368,18 +372,18 @@ private void writeBuf(ByteBuffer buf) throws IOException { 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); + appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(), + updatedPos); + length += partitionInfo.dataChannel.write(buf, updatedPos); } else { - length += partitionInfo.channel.write(buf); + 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 it's data to the merged file. Until + * 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. @@ -444,11 +448,13 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // 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)) { + Map 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; @@ -462,8 +468,8 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { deferredBufs = null; return; } - logger.trace("{} shuffleId {} reduceId {} onData writable", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId, + appShuffleId.shuffleId, reduceId); if (partitionInfo.getCurrentMapIndex() < 0) { partitionInfo.setCurrentMapIndex(mapIndex); } @@ -479,12 +485,12 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // 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.dataChannel.position(partitionInfo.getPosition() + length); partitionInfo.setEncounteredFailure(false); } } else { - logger.trace("{} shuffleId {} reduceId {} onData deferred", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + 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 @@ -516,8 +522,8 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { @Override public void onComplete(String streamId) throws IOException { - logger.trace("{} shuffleId {} reduceId {} onComplete invoked", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + 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 @@ -536,10 +542,11 @@ public void onComplete(String streamId) throws IOException { // TODO received from the driver? If so, then we need to know # maps for this shuffle. synchronized (partitionInfo) { + Map 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 (!partitions.containsKey(partitionId)) { + if (shufflePartitions == null || !shufflePartitions.containsKey(reduceId)) { deferredBufs = null; throw new RuntimeException(String.format("Block %s %s", msg.blockId, ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); @@ -591,10 +598,12 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { // 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)) { + Map shufflePartitions = partitions.get(appShuffleId); + if (isWriting && partitionInfo != null && shufflePartitions != null + && shufflePartitions.containsKey(reduceId)) { synchronized (partitionInfo) { - logger.debug("{} shuffleId {} reduceId {} set encountered failure", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId); + logger.debug("{} shuffleId {} reduceId {} set encountered failure", appShuffleId.appId, + appShuffleId.shuffleId, reduceId); partitionInfo.setCurrentMapIndex(-1); partitionInfo.setEncounteredFailure(true); } @@ -610,37 +619,31 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc List bitmaps = new LinkedList<>(); List reduceIds = new LinkedList<>(); List sizes = new LinkedList<>(); - Iterator> iterator = - partitions.entrySet().iterator(); - while (iterator.hasNext()) { - Map.Entry entry = iterator.next(); - AppShufflePartitionId partitionId = entry.getKey(); - AppShufflePartitionInfo partition = entry.getValue(); - if (partitionId.compareAppShuffleId(msg.appId, msg.shuffleId)) { + Map shufflePartitions = + partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId)); + if (shufflePartitions != null) { + for (AppShufflePartitionInfo partition : shufflePartitions.values()) { 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()); + partition.dataChannel.truncate(partition.getPosition()); if (partition.getPosition() != partition.getLastChunkOffset()) { partition.updateChunkInfo(partition.getPosition(), partition.lastMergedMapIndex); } bitmaps.add(partition.mapTracker); - reduceIds.add(partitionId.reduceId); + reduceIds.add(partition.reduceId); sizes.add(partition.getPosition()); } catch (IOException ioe) { logger.warn("Exception while finalizing shuffle partition {} {} {}", msg.appId, - msg.shuffleId, partitionId.reduceId, ioe); + msg.shuffleId, partition.reduceId, ioe); } finally { try { - partition.channel.close(); - partition.metaChannel.close(); - partition.indexWriteStream.close(); + partition.closeAllFiles(); } catch (IOException closeEx) { logger.warn("Exception while closing stream of shuffle partition {} {} {}", msg.appId, - msg.shuffleId, partitionId.reduceId, closeEx); + msg.shuffleId, partition.reduceId, closeEx); } } } @@ -672,19 +675,22 @@ public void registerExecutor(String appId, String[] 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 partition for an application. This is used to key the - * metadata tracked for each shuffle partition that's being actively merged. + * ID that uniquely identifies a shuffle for an application. This is used as a key in + * {@link #partitions}. */ - public static class AppShufflePartitionId { + public static class AppShuffleId { public final String appId; public final int shuffleId; - public final int reduceId; - AppShufflePartitionId(String appId, int shuffleId, int reduceId) { + AppShuffleId(String appId, int shuffleId) { this.appId = appId; this.shuffleId = shuffleId; - this.reduceId = reduceId; } @Override @@ -695,14 +701,13 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - AppShufflePartitionId that = (AppShufflePartitionId) o; - return shuffleId == that.shuffleId && reduceId == that.reduceId - && Objects.equal(appId, that.appId); + AppShuffleId that = (AppShuffleId) o; + return shuffleId == that.shuffleId && Objects.equal(appId, that.appId); } @Override public int hashCode() { - return Objects.hashCode(appId, shuffleId, reduceId); + return Objects.hashCode(appId, shuffleId); } @Override @@ -710,26 +715,17 @@ 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; + 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 @@ -740,10 +736,10 @@ public static class AppShufflePartitionInfo { 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; - 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; @@ -753,14 +749,15 @@ public static class AppShufflePartitionInfo { ByteBuf trackerBuf = null; AppShufflePartitionInfo( - AppShufflePartitionId partitionId, + AppShuffleId appShuffleId, + int reduceId, File dataFile, File indexFile, File metaFile) throws IOException { - this.partitionId = Preconditions.checkNotNull(partitionId, "partition id"); + this.appShuffleId = Preconditions.checkNotNull(appShuffleId, "app shuffle id"); + this.reduceId = reduceId; dataFile.createNewFile(); - this.dataFile = dataFile; - this.channel = new FileOutputStream(dataFile, true).getChannel(); + this.dataChannel = new FileOutputStream(dataFile, true).getChannel(); indexFile.createNewFile(); FileOutputStream fos = new FileOutputStream(indexFile, true); indexChannel = fos.getChannel(); @@ -781,8 +778,8 @@ public long getPosition() { } public void setPosition(long position) { - logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId, this.position, position); + logger.trace("{} shuffleId {} reduceId {} current pos {} update pos {}", appShuffleId.appId, + appShuffleId.shuffleId, reduceId, this.position, position); this.position = position; } @@ -800,7 +797,7 @@ int getCurrentMapIndex() { void setCurrentMapIndex(int mapIndex) { logger.trace("{} shuffleId {} reduceId {} updated mapIndex {} current mapIndex {}", - partitionId.appId, partitionId.shuffleId, partitionId.reduceId, currentMapIndex, mapIndex); + appShuffleId.appId, appShuffleId.shuffleId, reduceId, currentMapIndex, mapIndex); this.currentMapIndex = mapIndex; } @@ -809,8 +806,8 @@ long getLastChunkOffset() { } void blockMerged(int mapIndex) { - logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", partitionId.appId, - partitionId.shuffleId, partitionId.reduceId, mapIndex); + logger.debug("{} shuffleId {} reduceId {} updated merging mapIndex {}", appShuffleId.appId, + appShuffleId.shuffleId, reduceId, mapIndex); mapTracker.add(mapIndex); chunkTracker.add(mapIndex); lastMergedMapIndex = mapIndex; @@ -821,7 +818,6 @@ void resetChunkTracker() { } /** - * 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. * * @param chunkOffset the offset of the chunk in the data file. @@ -834,13 +830,14 @@ void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException { writeChunkTracker(mapIndex); idxStartPos = indexChannel.position(); logger.trace("{} shuffleId {} reduceId {} updated index current {} updated {}", - partitionId.appId, partitionId.shuffleId, partitionId.reduceId, this.lastChunkOffset, + 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("{} reset index to position {}", dataFile.getName(), idxStartPos); + logger.warn("{} shuffleId {} reduceId {} reset index to position {}", + appShuffleId.appId, appShuffleId.shuffleId, reduceId, idxStartPos); indexChannel.position(idxStartPos); } throw ioe; @@ -859,16 +856,25 @@ private void writeChunkTracker(int mapIndex) throws IOException { Encoders.Bitmaps.encode(trackerBuf, chunkTracker); long metaStartPos = metaChannel.position(); try { - logger.trace("{} write chunk to meta file", dataFile.getName()); + 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("{} reset position of meta file to {}", dataFile.getName(), metaStartPos); + 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(); + } } /** From 48ae819c878e3a60389b7689b7a519e15dac60dd Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 27 Oct 2020 23:03:16 -0700 Subject: [PATCH 22/35] Added support for subdirs under merge_manager dirs and removed the yarn dir pattern --- .../network/shuffle/ExternalBlockHandler.java | 10 +- .../shuffle/MergedShuffleFileManager.java | 23 +-- .../shuffle/RemoteBlockPushResolver.java | 86 +++++------ .../shuffle/ExternalBlockHandlerSuite.java | 2 +- .../shuffle/RemoteBlockPushResolverSuite.java | 134 ++++++++++-------- .../network/yarn/YarnShuffleService.java | 9 +- 6 files changed, 118 insertions(+), 146 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 321b25305c50..a2bf66882fe7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -175,7 +175,7 @@ protected void handleMessage( RegisterExecutor msg = (RegisterExecutor) msgObj; checkAuth(client, msg.appId); blockManager.registerExecutor(msg.appId, msg.execId, msg.executorInfo); - mergeManager.registerExecutor(msg.appId, msg.executorInfo.localDirs); + mergeManager.registerExecutor(msg.appId, msg.executorInfo); callback.onSuccess(ByteBuffer.wrap(new byte[0])); } finally { responseDelayContext.stop(); @@ -232,6 +232,7 @@ public StreamManager getStreamManager() { */ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { blockManager.applicationRemoved(appId, cleanupLocalDirs); + mergeManager.applicationRemoved(appId, cleanupLocalDirs); } /** @@ -444,12 +445,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } @Override - public void registerApplication(String appId, String user) { - // No-op. Do nothing. - } - - @Override - public void registerExecutor(String appId, String[] localDirs) { + public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { // No-Op. Do nothing. } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java index ef4dbb2bd005..500bcd6854e9 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/MergedShuffleFileManager.java @@ -21,6 +21,7 @@ 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; @@ -56,25 +57,15 @@ public interface MergedShuffleFileManager { MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException; /** - * Registers an application when it starts. It also stores the username which is necessary - * for generating the host local directories for merged shuffle files. - * Right now, this is invoked by YarnShuffleService. + * Registers an executor with MergedShuffleFileManager. This executor-info provides + * the directories and number of sub-dirs per dir so that MergedShuffleFileManager knows where to + * store and look for shuffle data for a given application. It is invoked by the RPC call when + * executor tries to register with the local shuffle service. * * @param appId application ID - * @param user username + * @param executorInfo The list of local dirs that this executor gets granted from NodeManager */ - void registerApplication(String appId, String user); - - /** - * Registers an executor with its local dir list when it starts. This provides the specific path - * so MergedShuffleFileManager knows where to store and look for shuffle data for a - * given application. It is invoked by the RPC call when executor tries to register with the - * local shuffle service. - * - * @param appId application ID - * @param localDirs The list of local dirs that this executor gets granted from NodeManager - */ - void registerExecutor(String appId, String[] localDirs); + void registerExecutor(String appId, ExecutorShuffleInfo executorInfo); /** * Invoked when an application finishes. This cleans up any remaining metadata associated with diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 1a7481cf422c..3887dbbd70b6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -37,6 +37,7 @@ 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; @@ -56,6 +57,7 @@ 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; @@ -70,7 +72,8 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { private static final Logger logger = LoggerFactory.getLogger(RemoteBlockPushResolver.class); - private static final String MERGE_MANAGER_DIR = "merge_manager"; + @VisibleForTesting + static final String MERGE_MANAGER_DIR = "merge_manager"; private final ConcurrentMap appsPathInfo; private final ConcurrentMap> partitions; @@ -78,14 +81,13 @@ public class RemoteBlockPushResolver implements MergedShuffleFileManager { 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 indexCache; @SuppressWarnings("UnstableApiUsage") - public RemoteBlockPushResolver(TransportConf conf, String relativeMergeDirPathPattern) { + public RemoteBlockPushResolver(TransportConf conf) { this.conf = conf; this.partitions = Maps.newConcurrentMap(); this.appsPathInfo = Maps.newConcurrentMap(); @@ -103,7 +105,6 @@ public ShuffleIndexInformation load(File file) throws IOException { .maximumWeight(conf.mergedIndexCacheSize()) .weigher((Weigher) (file, indexInfo) -> indexInfo.getSize()) .build(indexCacheLoader); - this.relativeMergeDirPathPattern = relativeMergeDirPathPattern; this.errorHandler = new ErrorHandler.BlockPushErrorHandler(); } @@ -133,11 +134,12 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( } } catch (IOException e) { logger.error( - "Cannot create merged shuffle partition {} with shuffle file {}, index file {}, and " - + "meta file {}", key, indexFile.getAbsolutePath(), + "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 %s", key.toString()), e); + String.format("Cannot initialize merged shuffle partition for appId %s shuffleId %s " + + "reduceId %s", appShuffleId.appId, appShuffleId.shuffleId, reduceId), e); } }); } @@ -192,31 +194,16 @@ public ManagedBuffer getMergedBlockData(String appId, int shuffleId, int reduceI * 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); + File targetFile = ExecutorDiskUtils.getFile(appPathsInfo.activeLocalDirs, + appPathsInfo.subDirsPerLocalDir, 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); @@ -236,12 +223,10 @@ private File getMergedShuffleMetaFile(AppShuffleId appShuffleId, int reduceId) { 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, + String[] activeLocalDirs = Preconditions.checkNotNull(appPathsInfo.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); + return activeLocalDirs; } @Override @@ -268,17 +253,17 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } } if (cleanupLocalDirs) { - Path[] dirs = Arrays.stream(getActiveLocalDirs(appPathsInfo.activeLocalDirs)) - .map(dir -> dir.resolve(getRelativePath(appPathsInfo.user, appId))) - .toArray(Path[]::new); + Path[] dirs = Arrays.stream(appPathsInfo.activeLocalDirs) + .map(dir -> Paths.get(dir)).toArray(Path[]::new); directoryCleaner.execute(() -> deleteExecutorDirs(dirs)); } } /** - * Synchronously delete local dirs, executed in a separate thread. + * Serially delete local dirs, executed in a separate thread. */ - private void deleteExecutorDirs(Path[] dirs) { + @VisibleForTesting + void deleteExecutorDirs(Path[] dirs) { for (Path localDir : dirs) { try { if (Files.exists(localDir)) { @@ -655,23 +640,17 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } @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) { + public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { if (logger.isDebugEnabled()) { - logger.debug("register executor with RemoteBlockPushResolver {} {}", - appId, Arrays.toString(localDirs)); + logger.debug("register executor with RemoteBlockPushResolver {} local-dirs {} " + + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs), + executorInfo.subDirsPerLocalDir); } - Preconditions.checkNotNull(appsPathInfo.get(appId), - "application " + appId + " is not registered or NM was restarted."); + appsPathInfo.putIfAbsent(appId, new AppPathsInfo()); appsPathInfo.compute(appId, (targetAppId, appPathsInfo) -> { assert appPathsInfo != null; - return appPathsInfo.updateActiveLocalDirs( - targetAppId, relativeMergeDirPathPattern, localDirs); + return appPathsInfo.updateActiveLocalDirs(targetAppId, executorInfo.localDirs, + executorInfo.subDirsPerLocalDir); }); } @@ -882,20 +861,19 @@ void closeAllFiles() throws IOException { */ private static class AppPathsInfo { - private final String user; private String[] activeLocalDirs; - - AppPathsInfo(String user) { - this.user = Preconditions.checkNotNull(user, "user cannot be null"); - } + private int subDirsPerLocalDir; private AppPathsInfo updateActiveLocalDirs( - String appId, String relativePathPattern, String[] localDirs) { + String appId, + String[] localDirs, + int subDirsPerLocalDir) { if (activeLocalDirs == null) { - String relativePath = String.format(relativePathPattern, user, appId); activeLocalDirs = Arrays.stream(localDirs) - .map(localDir -> localDir.substring(0, localDir.indexOf(relativePath))) + .map(localDir -> + Paths.get(localDir).getParent().resolve(MERGE_MANAGER_DIR).toFile().getPath()) .toArray(String[]::new); + this.subDirsPerLocalDir = subDirsPerLocalDir; logger.info("Updated the active local dirs {} for application {}", Arrays.toString(activeLocalDirs), appId); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java index 680b8d74a2ee..f06e7cb047f1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java @@ -77,7 +77,7 @@ public void testRegisterExecutor() { ByteBuffer registerMessage = new RegisterExecutor("app0", "exec1", config).toByteBuffer(); handler.receive(client, registerMessage, callback); verify(blockResolver, times(1)).registerExecutor("app0", "exec1", config); - verify(mergedShuffleManager, times(1)).registerExecutor("app0", localDirs); + verify(mergedShuffleManager, times(1)).registerExecutor("app0", config); verify(callback, times(1)).onSuccess(any(ByteBuffer.class)); verify(callback, never()).onFailure(any(Throwable.class)); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index f74648f6149e..583d1f764905 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -21,8 +21,10 @@ 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; @@ -30,6 +32,7 @@ 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; @@ -40,6 +43,7 @@ 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.PushBlockStream; import org.apache.spark.network.util.MapConfigProvider; @@ -51,32 +55,29 @@ 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; + private Path[] localDirs; @Before public void before() throws IOException { - localDirs = new String[]{Paths.get("target/l1").toAbsolutePath().toString(), - Paths.get("target/l2").toAbsolutePath().toString()}; - cleanupLocalDirs(); + localDirs = createLocalDirs(2); 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)); + pushResolver = new RemoteBlockPushResolver(conf); + registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); } @After public void after() { try { - cleanupLocalDirs(); + for (Path local : localDirs) { + FileUtils.deleteDirectory(local.toFile()); + } removeApplication(TEST_APP); } catch (Exception e) { // don't fail if clean up doesn't succeed. @@ -84,12 +85,6 @@ public void after() { } } - private void cleanupLocalDirs() throws IOException { - for (String local : localDirs) { - FileUtils.deleteDirectory(new File(local)); - } - } - @Test(expected = RuntimeException.class) public void testNoIndexFile() { try { @@ -102,34 +97,27 @@ public void testNoIndexFile() { @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), - }; + String[] pushBlockIds = new String[] {"shuffle_0_0_0", "shuffle_0_1_0"}; ByteBuffer[] blocks = new ByteBuffer[]{ ByteBuffer.wrap(new byte[4]), ByteBuffer.wrap(new byte[5]) }; - pushBlockHelper(TEST_APP, pushBlocks, blocks); + pushBlockHelper(TEST_APP, pushBlockIds, 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), - }; + String[] pushBlockIds = + new String[] {"shuffle_0_0_0", "shuffle_0_1_0", "shuffle_0_2_0", "shuffle_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); + pushBlockHelper(TEST_APP, pushBlockIds, 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}}); } @@ -310,7 +298,7 @@ public void testTooLateArrival() throws IOException { @Test public void testIncompleteStreamsAreOverwritten() throws IOException { - registerExecutor(TEST_APP, prepareBlockManagerLocalDirs(TEST_APP, TEST_USER, localDirs)); + registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); StreamCallbackWithID stream1 = pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0)); @@ -371,23 +359,18 @@ public void testFailureWith3Streams() throws IOException { @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)); + Path[] activeLocalDirs = createLocalDirs(1); + registerExecutor(testApp, prepareLocalDirs(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); + 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( - "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")); + activeLocalDirs[0].toFile().getPath())); removeApplication(testApp); try { pushResolver.getMergedBlockDirs(testApp); @@ -398,25 +381,56 @@ public void testUpdateLocalDirsOnlyOnce() throws IOException { } } - /** - * Registers the app with RemoteBlockPushResolver. - */ - private void registerApplication(String appId, String user) throws IOException { - pushResolver.registerApplication(appId, user); + @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)); + String[] pushBlockIds = new String[] {"shuffle_0_0_0"}; + ByteBuffer[] blocks = new ByteBuffer[] {ByteBuffer.wrap(new byte[4])}; + pushBlockHelper(testApp, pushBlockIds, blocks); + 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 void registerExecutor(String appId, String[] localDirs) throws IOException { - pushResolver.registerExecutor(appId, localDirs); - for (String localDir : pushResolver.getMergedBlockDirs(appId)) { - Files.createDirectories(Paths.get(localDir)); + 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 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 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) { @@ -448,12 +462,12 @@ private void validateChunks( private void pushBlockHelper( String appId, - PushBlockStream[] pushBlocks, + String[] blockIds, ByteBuffer[] blocks) throws IOException { - Preconditions.checkArgument(pushBlocks.length == blocks.length); - for (int i = 0; i < pushBlocks.length; i++) { + Preconditions.checkArgument(blockIds.length == blocks.length); + for (int i = 0; i < blockIds.length; i++) { StreamCallbackWithID stream = pushResolver.receiveBlockDataAsStream( - new PushBlockStream(appId, pushBlocks[i].blockId, 0)); + new PushBlockStream(appId, blockIds[i], 0)); stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index b1fe64077558..a3650d8fdea1 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -95,9 +95,6 @@ public class YarnShuffleService extends AuxiliaryService { 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/"; - // just for testing when you want to find an open port @VisibleForTesting static int boundPort = -1; @@ -130,8 +127,6 @@ public class YarnShuffleService extends AuxiliaryService { // Handles registering executors and opening shuffle blocks @VisibleForTesting ExternalBlockHandler blockHandler; - @VisibleForTesting - RemoteBlockPushResolver shuffleMergeManager; // Where to store & reload executor info for recovering state after an NM restart @VisibleForTesting @@ -178,7 +173,7 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - shuffleMergeManager = new RemoteBlockPushResolver(transportConf, APP_BASE_RELATIVE_PATH); + RemoteBlockPushResolver shuffleMergeManager = new RemoteBlockPushResolver(transportConf); blockHandler = new ExternalBlockHandler( transportConf, registeredExecutorFile, shuffleMergeManager); @@ -287,7 +282,6 @@ public void initializeApplication(ApplicationInitializationContext context) { } catch (Exception e) { logger.error("Exception when initializing application {}", appId, e); } - shuffleMergeManager.registerApplication(appId, context.getUser()); } @Override @@ -306,7 +300,6 @@ public void stopApplication(ApplicationTerminationContext context) { secretManager.unregisterApp(appId); } blockHandler.applicationRemoved(appId, false /* clean up local dirs */); - shuffleMergeManager.applicationRemoved(appId, false); } catch (Exception e) { logger.error("Exception when stopping application {}", appId, e); } From 9b031f766677a8979ef118e62f42eedb9c5e13ac Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 28 Oct 2020 22:58:25 -0700 Subject: [PATCH 23/35] Addressed test failure and other review comments in RemoteBlockPushResolver --- .../network/shuffle/ExternalBlockHandler.java | 2 +- .../shuffle/RemoteBlockPushResolver.java | 694 ++++++++++-------- 2 files changed, 387 insertions(+), 309 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index a2bf66882fe7..296287d748b5 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -451,7 +451,7 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { @Override public void applicationRemoved(String appId, boolean cleanupLocalDirs) { - throw new UnsupportedOperationException("Cannot handle shuffle block merge"); + // No-Op. Do nothing. } @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 3887dbbd70b6..44987ceca7ba 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -28,6 +28,7 @@ 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; @@ -45,8 +46,6 @@ 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; @@ -243,12 +242,7 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { 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); - } + partitionInfo.closeAllFiles(); } } } @@ -328,315 +322,85 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { // 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 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); - } + 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; } - } - /** - * 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 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 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) throws IOException { - logger.trace("{} shuffleId {} reduceId {} onComplete invoked", appShuffleId.appId, - appShuffleId.shuffleId, reduceId); - if (partitionInfo == null) { + @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)); - } 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 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)); } + // For duplicate block that is received before the shuffle merge finalizes, the + // server should respond success to the client. } - 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 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); - } + @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); - List bitmaps = new LinkedList<>(); - List reduceIds = new LinkedList<>(); - List sizes = new LinkedList<>(); Map 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); - } + 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 partitions = shufflePartitions.values(); + int totalPartitions = partitions.size(); + RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions]; + int[] reduceIds = new int[totalPartitions]; + long[] sizes = new long[totalPartitions]; + Iterator partitionsIter = shufflePartitions.values().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.toArray(new RoaringBitmap[bitmaps.size()]), - Ints.toArray(reduceIds), Longs.toArray(sizes)); + return new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes); } @Override @@ -659,6 +423,280 @@ private static String generateFileName(AppShuffleId appShuffleId, int reduceId) reduceId); } + /** + * Callback for push stream that handles blocks which are not already merged. + */ + static class StreamCallbackImpl 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 deferredBufs; + + private StreamCallbackImpl( + 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.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 {}", + 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 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 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.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() >= 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.blockId)); + } + } + 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 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}. @@ -704,7 +742,7 @@ public static class AppShufflePartitionInfo { private final AppShuffleId appShuffleId; private final int reduceId; // The merged shuffle data file channel - public final FileChannel dataChannel; + 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 @@ -714,11 +752,11 @@ public static class AppShufflePartitionInfo { // 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; + 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 final FileChannel metaChannel; + 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; @@ -735,14 +773,11 @@ public static class AppShufflePartitionInfo { 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); + this.dataChannel = new FileOutputStream(dataFile).getChannel(); + FileOutputStream fos = new FileOutputStream(indexFile); indexChannel = fos.getChannel(); this.indexWriteStream = new DataOutputStream(new BufferedOutputStream(fos)); - metaFile.createNewFile(); - metaChannel = new FileOutputStream(metaFile, true).getChannel(); + metaChannel = new FileOutputStream(metaFile).getChannel(); this.currentMapIndex = -1; // Writing 0 offset so that we can reuse ShuffleIndexInformation.getIndex() updateChunkInfo(0L, -1); @@ -848,11 +883,48 @@ private void writeChunkTracker(int mapIndex) throws IOException { } } - void closeAllFiles() throws IOException { - this.dataChannel.close(); - this.metaChannel.close(); - this.indexWriteStream.close(); - this.indexChannel.close(); + 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(); } } @@ -871,11 +943,17 @@ private AppPathsInfo updateActiveLocalDirs( 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; - logger.info("Updated the active local dirs {} for application {}", - Arrays.toString(activeLocalDirs), appId); + if (logger.isInfoEnabled()) { + logger.info("Updated the active local dirs {} for application {}", + Arrays.toString(activeLocalDirs), appId); + } } return this; } From 807cc7b506054cd5f51ad30ee977493bbd5cb2c7 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 29 Oct 2020 10:14:33 -0700 Subject: [PATCH 24/35] Minor change in finalization --- .../apache/spark/network/shuffle/RemoteBlockPushResolver.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 44987ceca7ba..0d5989a62dd6 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -374,7 +374,7 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions]; int[] reduceIds = new int[totalPartitions]; long[] sizes = new long[totalPartitions]; - Iterator partitionsIter = shufflePartitions.values().iterator(); + Iterator partitionsIter = partitions.iterator(); int idx = 0; while (partitionsIter.hasNext()) { AppShufflePartitionInfo partition = partitionsIter.next(); From 5b169bcbf86f7046fb8ac28231b18df610540fa9 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 29 Oct 2020 17:55:05 -0700 Subject: [PATCH 25/35] Removing the partition from inner map after the files are closed --- .../shuffle/RemoteBlockPushResolver.java | 6 ++- .../shuffle/RemoteBlockPushResolverSuite.java | 41 +++++++++++++++---- 2 files changed, 38 insertions(+), 9 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 0d5989a62dd6..cb6ccb8c52d3 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -378,7 +378,6 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc 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 @@ -390,12 +389,15 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOExc } bitmaps[idx] = partition.mapTracker; reduceIds[idx] = partition.reduceId; - sizes[idx] = partition.getPosition(); + 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 from shuffle maps after the files are written so that + // any new stream for the same reduce partition will see that the data file exists. + partitionsIter.remove(); } } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 583d1f764905..e9273170a751 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -45,6 +45,7 @@ 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; @@ -103,6 +104,9 @@ public void testBasicBlockMerge() throws IOException { ByteBuffer.wrap(new byte[5]) }; pushBlockHelper(TEST_APP, pushBlockIds, 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}}); } @@ -118,10 +122,31 @@ public void testDividingMergedBlocksIntoChunks() throws IOException { ByteBuffer.wrap(new byte[3]) }; pushBlockHelper(TEST_APP, pushBlockIds, 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 { + String[] pushBlockIds = + new String[] {"shuffle_0_0_0", "shuffle_0_1_0", "shuffle_0_0_1", "shuffle_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, pushBlockIds, 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 { PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); @@ -142,7 +167,6 @@ public void testDeferredBufsAreWrittenDuringOnData() throws IOException { // 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}}); @@ -168,7 +192,6 @@ public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException { // 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}}); @@ -216,7 +239,6 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOE // 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}}); @@ -229,7 +251,6 @@ public void testFailureAfterData() throws IOException { 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()); @@ -244,7 +265,6 @@ public void testFailureAfterMultipleDataBlocks() throws IOException { 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()); @@ -260,7 +280,6 @@ public void testFailureAfterComplete() throws IOException { 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}}); @@ -397,6 +416,7 @@ void deleteExecutorDirs(Path[] dirs) { String[] pushBlockIds = new String[] {"shuffle_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); @@ -439,6 +459,14 @@ private void removeApplication(String appId) { 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, @@ -471,6 +499,5 @@ private void pushBlockHelper( stream.onData(stream.getID(), blocks[i]); stream.onComplete(stream.getID()); } - pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(appId, 0)); } } From 9ece58781034ee09f28b742e3e0b822ce34a2d3f Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Thu, 29 Oct 2020 18:39:23 -0700 Subject: [PATCH 26/35] Server side configuration to specify the implementation of MergedShuffleFileManager --- .../spark/network/util/TransportConf.java | 12 ++++ .../network/shuffle/ExternalBlockHandler.java | 11 +++- .../shuffle/RemoteBlockPushResolver.java | 7 ++- .../network/yarn/YarnShuffleService.java | 23 ++++++- .../network/yarn/YarnShuffleServiceSuite.java | 60 +++++++++++++++++++ 5 files changed, 105 insertions(+), 8 deletions(-) create mode 100644 common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 196eb776d61b..b1309d6ec9e1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -363,6 +363,18 @@ 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"); + } + /** * The minimum size of a chunk when dividing a merged shuffle file into multiple chunks during * push-based shuffle. diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java index 296287d748b5..be0a427505b7 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java @@ -68,7 +68,7 @@ public ExternalBlockHandler(TransportConf conf, File registeredExecutorFile) throws IOException { this(new OneForOneStreamManager(), new ExternalShuffleBlockResolver(conf, registeredExecutorFile), - new NoOpMergedShuffleFileManager()); + new NoOpMergedShuffleFileManager(conf)); } public ExternalBlockHandler( @@ -89,7 +89,7 @@ public ExternalShuffleBlockResolver getBlockResolver() { public ExternalBlockHandler( OneForOneStreamManager streamManager, ExternalShuffleBlockResolver blockManager) { - this(streamManager, blockManager, new NoOpMergedShuffleFileManager()); + this(streamManager, blockManager, new NoOpMergedShuffleFileManager(null)); } /** Enables mocking out the StreamManager, BlockManager, and MergeManager. */ @@ -432,7 +432,12 @@ public ManagedBuffer next() { * Dummy implementation of merged shuffle file manager. Suitable for when push-based shuffle * is not enabled. */ - private static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager { + public static class NoOpMergedShuffleFileManager implements MergedShuffleFileManager { + + // This constructor is needed because we use this constructor to instantiate an implementation + // of MergedShuffleFileManager using reflection. + // See YarnShuffleService#newMergedShuffleFileManagerInstance. + public NoOpMergedShuffleFileManager(TransportConf transportConf) {} @Override public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index cb6ccb8c52d3..53dd8b8bc374 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -323,7 +323,8 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { final AppShufflePartitionInfo partitionInfo = partitionInfoBeforeCheck != null && partitionInfoBeforeCheck.mapTracker.contains(mapIndex) ? null : partitionInfoBeforeCheck; if (partitionInfo != null) { - return new StreamCallbackImpl(this, msg, appShuffleId, reduceId, mapIndex, partitionInfo); + return new PushBlockStreamCallback( + 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. @@ -428,7 +429,7 @@ private static String generateFileName(AppShuffleId appShuffleId, int reduceId) /** * Callback for push stream that handles blocks which are not already merged. */ - static class StreamCallbackImpl implements StreamCallbackWithID { + static class PushBlockStreamCallback implements StreamCallbackWithID { private final RemoteBlockPushResolver mergeManager; private final PushBlockStream msg; @@ -445,7 +446,7 @@ static class StreamCallbackImpl implements StreamCallbackWithID { // Use on-heap instead of direct ByteBuffer since these buffers will be GC'ed very quickly private List deferredBufs; - private StreamCallbackImpl( + private PushBlockStreamCallback( RemoteBlockPushResolver mergeManager, PushBlockStream msg, AppShuffleId appShuffleId, diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index a3650d8fdea1..5b7f37f01d01 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -41,6 +41,8 @@ import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.*; +import org.apache.spark.network.shuffle.MergedShuffleFileManager; +import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.LevelDBProvider; import org.iq80.leveldb.DB; import org.iq80.leveldb.DBIterator; @@ -54,7 +56,6 @@ import org.apache.spark.network.server.TransportServer; import org.apache.spark.network.server.TransportServerBootstrap; import org.apache.spark.network.shuffle.ExternalBlockHandler; -import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.TransportConf; import org.apache.spark.network.yarn.util.HadoopConfigProvider; @@ -173,7 +174,8 @@ protected void serviceInit(Configuration conf) throws Exception { } TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); - RemoteBlockPushResolver shuffleMergeManager = new RemoteBlockPushResolver(transportConf); + MergedShuffleFileManager shuffleMergeManager = newMergedShuffleFileManagerInstance( + transportConf); blockHandler = new ExternalBlockHandler( transportConf, registeredExecutorFile, shuffleMergeManager); @@ -222,6 +224,23 @@ protected void serviceInit(Configuration conf) throws Exception { } } + @VisibleForTesting + static MergedShuffleFileManager newMergedShuffleFileManagerInstance(TransportConf conf) { + String mergeManagerImplClassName = conf.mergeShuffleFileManagerImpl(); + try { + Class mergeManagerImplClazz = Class.forName( + mergeManagerImplClassName, true, Thread.currentThread().getContextClassLoader()); + Class mergeManagerSubClazz = + mergeManagerImplClazz.asSubclass(MergedShuffleFileManager.class); + // The assumption is that all the custom implementations just like the RemoteBlockPushResolver + // will also need the transport configuration. + return mergeManagerSubClazz.getConstructor(TransportConf.class).newInstance(conf); + } catch (Exception e) { + logger.error("Unable to create an instance of {}", mergeManagerImplClassName); + return new RemoteBlockPushResolver(conf); + } + } + private void loadSecretsFromDb() throws IOException { secretsFile = initRecoveryDb(SECRETS_RECOVERY_FILE_NAME); diff --git a/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java b/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java new file mode 100644 index 000000000000..0fb8f038b49d --- /dev/null +++ b/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java @@ -0,0 +1,60 @@ +/* + * 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.yarn; + +import org.apache.spark.network.shuffle.ExternalBlockHandler; +import org.apache.spark.network.shuffle.MergedShuffleFileManager; +import org.apache.spark.network.shuffle.RemoteBlockPushResolver; +import org.apache.spark.network.util.TransportConf; +import org.junit.Test; + +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +public class YarnShuffleServiceSuite { + + @Test + public void testCreateDefaultMergedShuffleFileManagerInstance() { + TransportConf mockConf = mock(TransportConf.class); + when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( + "org.apache.spark.network.shuffle.RemoteBlockPushResolver"); + MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( + mockConf); + assertTrue(mergeMgr instanceof RemoteBlockPushResolver); + } + + @Test + public void testCreateNoOpMergedShuffleFileManagerInstance() { + TransportConf mockConf = mock(TransportConf.class); + when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( + "org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager"); + MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( + mockConf); + assertTrue(mergeMgr instanceof ExternalBlockHandler.NoOpMergedShuffleFileManager); + } + + @Test + public void testInvalidClassNameOfMergeManagerWillUseRemoteBlockPushResolverImpl() { + TransportConf mockConf = mock(TransportConf.class); + when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( + "org.apache.spark.network.shuffle.NotExistent"); + MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( + mockConf); + assertTrue(mergeMgr instanceof RemoteBlockPushResolver); + } +} From d13c7ade5e4ae7e8d319a82523a004c4cf36e7da Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Sun, 1 Nov 2020 16:02:46 -0800 Subject: [PATCH 27/35] Change the Push block stream to not encode shuffle Id, map index, and reduceId in a string --- .../network/shuffle/OneForOneBlockPusher.java | 9 +- .../shuffle/RemoteBlockPushResolver.java | 26 ++-- .../shuffle/protocol/PushBlockStream.java | 38 ++++-- .../shuffle/OneForOneBlockPusherSuite.java | 66 ++++----- .../shuffle/RemoteBlockPushResolverSuite.java | 125 ++++++++---------- 5 files changed, 133 insertions(+), 131 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java index 407b248170a4..563f5dbec06e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -115,7 +115,14 @@ public void start() { for (int i = 0; i < blockIds.length; i++) { assert buffers.containsKey(blockIds[i]) : "Could not find the block buffer for block " + blockIds[i]; - ByteBuffer header = new PushBlockStream(appId, blockIds[i], i).toByteBuffer(); + String[] blockIdParts = blockIds[i].split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals( + PushBlockStream.SHUFFLE_PUSH_BLOCK_PREFIX)) { + throw new IllegalArgumentException( + "Unexpected shuffle push block id format: " + blockIds[i]); + } + ByteBuffer header = new PushBlockStream(appId, Integer.parseInt(blockIdParts[1]), + Integer.parseInt(blockIdParts[2]), Integer.parseInt(blockIdParts[3]) , i).toByteBuffer(); client.uploadStream(new NioManagedBuffer(header), buffers.get(blockIds[i]), new BlockPushCallback(i, blockIds[i])); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 53dd8b8bc374..b2ed9164b7ed 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -273,16 +273,9 @@ void deleteExecutorDirs(Path[] dirs) { @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]); + AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId); AppShufflePartitionInfo partitionInfoBeforeCheck = - getOrCreateAppShufflePartitionInfo(appShuffleId, reduceId); - + 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. @@ -321,17 +314,18 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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; + && partitionInfoBeforeCheck.mapTracker.contains(msg.mapIndex) ? null + : partitionInfoBeforeCheck; if (partitionInfo != null) { return new PushBlockStreamCallback( - this, msg, appShuffleId, reduceId, mapIndex, partitionInfo); + 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.blockId; + return msg.streamId; } @Override @@ -345,7 +339,7 @@ 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, + 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 @@ -463,7 +457,7 @@ private PushBlockStreamCallback( @Override public String getID() { - return msg.blockId; + return msg.streamId; } /** @@ -635,7 +629,7 @@ public void onComplete(String streamId) throws IOException { // 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, + throw new RuntimeException(String.format("Block %s %s", msg.streamId, ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); } // Check if we can commit this block @@ -668,7 +662,7 @@ public void onComplete(String streamId) throws IOException { deferredBufs = null; throw new RuntimeException(String.format("%s %s to merged shuffle", ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX, - msg.blockId)); + msg.streamId)); } } isWriting = false; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index 7eab5a644783..0521e47ceddb 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -23,7 +23,6 @@ import org.apache.spark.network.protocol.Encoders; // Needed by ScalaDoc. See SPARK-7726 -import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; /** @@ -31,16 +30,24 @@ * 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; - public PushBlockStream(String appId, String blockId, int index) { + public PushBlockStream(String appId, int shuffleId, int mapIndex, int reduceId, int index) { this.appId = appId; - this.blockId = blockId; + this.shuffleId = shuffleId; + this.mapIndex = mapIndex; + this.reduceId = reduceId; this.index = index; + this.streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX, + shuffleId, mapIndex, reduceId); } @Override @@ -50,14 +57,16 @@ protected Type type() { @Override public int hashCode() { - return Objects.hashCode(appId, blockId, index); + return Objects.hashCode(appId, shuffleId, mapIndex , reduceId, index); } @Override public String toString() { return Objects.toStringHelper(this) .add("appId", appId) - .add("blockId", blockId) + .add("streamId", shuffleId) + .add("mapIndex", mapIndex) + .add("reduceId", reduceId) .add("index", index) .toString(); } @@ -67,7 +76,9 @@ public boolean equals(Object other) { if (other != null && other instanceof PushBlockStream) { PushBlockStream o = (PushBlockStream) other; return Objects.equal(appId, o.appId) - && Objects.equal(blockId, o.blockId) + && shuffleId == o.shuffleId + && mapIndex == o.mapIndex + && reduceId == o.reduceId && index == o.index; } return false; @@ -75,21 +86,24 @@ public boolean equals(Object other) { @Override public int encodedLength() { - return Encoders.Strings.encodedLength(appId) - + Encoders.Strings.encodedLength(blockId) + 4; + return Encoders.Strings.encodedLength(appId) + 16; } @Override public void encode(ByteBuf buf) { Encoders.Strings.encode(buf, appId); - Encoders.Strings.encode(buf, blockId); + buf.writeInt(shuffleId); + buf.writeInt(mapIndex); + buf.writeInt(reduceId); buf.writeInt(index); } public static PushBlockStream decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); - String blockId = Encoders.Strings.decode(buf); + int streamId = buf.readInt(); + int mapIdx = buf.readInt(); + int reduceId = buf.readInt(); int index = buf.readInt(); - return new PushBlockStream(appId, blockId, index); + return new PushBlockStream(appId, streamId, mapIdx, reduceId, index); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java index ebcdba72aa1a..46a0f6cf420e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockPusherSuite.java @@ -45,77 +45,77 @@ public class OneForOneBlockPusherSuite { @Test public void testPushOne() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); + blocks.put("shufflePush_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[1]))); String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( blocks, blockIds, - Arrays.asList(new PushBlockStream("app-id", "shuffle_0_0_0", 0))); + Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0))); - verify(listener).onBlockFetchSuccess(eq("shuffle_0_0_0"), any()); + verify(listener).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); } @Test public void testPushThree() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); - blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + blocks.put("shufflePush_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shufflePush_0_1_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("shufflePush_0_2_0", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( blocks, blockIds, - Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); + Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), + new PushBlockStream("app-id", 0, 1, 0, 1), + new PushBlockStream("app-id", 0, 2, 0, 2))); - for (int i = 0; i < 3; i ++) { - verify(listener, times(1)).onBlockFetchSuccess(eq("b" + i), any()); - } + verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); + verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_1_0"), any()); + verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_2_0"), any()); } @Test public void testServerFailures() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); - blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("shufflePush_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shufflePush_0_1_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("shufflePush_0_2_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( blocks, blockIds, - Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); + Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), + new PushBlockStream("app-id", 0, 1, 0, 1), + new PushBlockStream("app-id", 0, 2, 0, 2))); - verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); - verify(listener, times(1)).onBlockFetchFailure(eq("b2"), any()); + verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("shufflePush_0_1_0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("shufflePush_0_2_0"), any()); } @Test public void testHandlingRetriableFailures() { LinkedHashMap blocks = Maps.newLinkedHashMap(); - blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); - blocks.put("b1", null); - blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + blocks.put("shufflePush_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shufflePush_0_1_0", null); + blocks.put("shufflePush_0_2_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); BlockFetchingListener listener = pushBlocks( blocks, blockIds, - Arrays.asList(new PushBlockStream("app-id", "b0", 0), - new PushBlockStream("app-id", "b1", 1), - new PushBlockStream("app-id", "b2", 2))); - - verify(listener, times(1)).onBlockFetchSuccess(eq("b0"), any()); - verify(listener, times(0)).onBlockFetchSuccess(not(eq("b0")), any()); - verify(listener, times(0)).onBlockFetchFailure(eq("b0"), any()); - verify(listener, times(1)).onBlockFetchFailure(eq("b1"), any()); - verify(listener, times(2)).onBlockFetchFailure(eq("b2"), any()); + Arrays.asList(new PushBlockStream("app-id", 0, 0, 0, 0), + new PushBlockStream("app-id", 0, 1, 0, 1), + new PushBlockStream("app-id", 0, 2, 0, 2))); + + verify(listener, times(1)).onBlockFetchSuccess(eq("shufflePush_0_0_0"), any()); + verify(listener, times(0)).onBlockFetchSuccess(not(eq("shufflePush_0_0_0")), any()); + verify(listener, times(0)).onBlockFetchFailure(eq("shufflePush_0_0_0"), any()); + verify(listener, times(1)).onBlockFetchFailure(eq("shufflePush_0_1_0"), any()); + verify(listener, times(2)).onBlockFetchFailure(eq("shufflePush_0_2_0"), any()); } /** diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index e9273170a751..79f2957b0cc1 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -98,12 +98,12 @@ public void testNoIndexFile() { @Test public void testBasicBlockMerge() throws IOException { - String[] pushBlockIds = new String[] {"shuffle_0_0_0", "shuffle_0_1_0"}; + 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, pushBlockIds, blocks); + pushBlockHelper(TEST_APP, pushBlocks, blocks); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0}, new long[] {9}); @@ -113,15 +113,19 @@ public void testBasicBlockMerge() throws IOException { @Test public void testDividingMergedBlocksIntoChunks() throws IOException { - String[] pushBlockIds = - new String[] {"shuffle_0_0_0", "shuffle_0_1_0", "shuffle_0_2_0", "shuffle_0_3_0"}; + 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, pushBlockIds, buffers); + pushBlockHelper(TEST_APP, pushBlocks, buffers); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0}, new long[] {13}); @@ -131,15 +135,18 @@ public void testDividingMergedBlocksIntoChunks() throws IOException { @Test public void testFinalizeWithMultipleReducePartitions() throws IOException { - String[] pushBlockIds = - new String[] {"shuffle_0_0_0", "shuffle_0_1_0", "shuffle_0_0_1", "shuffle_0_1_1"}; + 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, pushBlockIds, buffers); + pushBlockHelper(TEST_APP, pushBlocks, buffers); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0, 1}, new long[] {5, 8}); @@ -149,21 +156,16 @@ public void testFinalizeWithMultipleReducePartitions() throws IOException { @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)); + 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, "shuffle_0_1_0", 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + 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()); @@ -174,22 +176,17 @@ public void testDeferredBufsAreWrittenDuringOnData() throws IOException { @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)); + 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, "shuffle_0_1_0", 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + 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)); @@ -199,21 +196,17 @@ public void testDeferredBufsAreWrittenDuringOnComplete() throws IOException { @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)); + 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()); - - PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + 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}}); @@ -221,22 +214,17 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamHasCompleted() throws IOE @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)); + 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, "shuffle_0_0_0", 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + 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)); @@ -246,9 +234,8 @@ public void testDuplicateBlocksAreIgnoredWhenPrevStreamIsInProgress() throws IOE @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)); + 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)); @@ -258,9 +245,8 @@ public void testFailureAfterData() throws IOException { @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)); + 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])); @@ -272,9 +258,8 @@ public void testFailureAfterMultipleDataBlocks() throws IOException { @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)); + 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])); @@ -287,28 +272,26 @@ public void testFailureAfterComplete() throws IOException { @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)); + 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, pushBlocks[1].blockId, 0)); + 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 shuffle_0_1_0 received after merged shuffle is finalized", re.getMessage()); + "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; @@ -318,16 +301,13 @@ public void testTooLateArrival() throws IOException { @Test public void testIncompleteStreamsAreOverwritten() throws IOException { registerExecutor(TEST_APP, prepareLocalDirs(localDirs)); - PushBlockStream pbStream1 = new PushBlockStream(TEST_APP, "shuffle_0_0_0", 0); StreamCallbackWithID stream1 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream1.blockId, 0)); + 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")); - - PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, "shuffle_0_1_0", 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); + 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)); @@ -337,20 +317,16 @@ public void testIncompleteStreamsAreOverwritten() throws IOException { @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)); + 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, "shuffle_0_1_0", 0); + PushBlockStream pbStream2 = new PushBlockStream(TEST_APP, 0, 1, 0, 0); StreamCallbackWithID stream2 = - pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, pbStream2.blockId, 0)); - // There is a failure + pushResolver.receiveBlockDataAsStream(new PushBlockStream(TEST_APP, 0, 1, 0, 0)); + // There is a failure with stream2 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)); + 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 @@ -359,7 +335,7 @@ public void testFailureWith3Streams() throws IOException { stream3.onComplete(stream2.getID()); } catch (RuntimeException re) { assertEquals( - "Couldn't find an opportunity to write block shuffle_0_2_0 to merged shuffle", + "Couldn't find an opportunity to write block shufflePush_0_2_0 to merged shuffle", re.getMessage()); failedEx = re; } @@ -413,7 +389,7 @@ void deleteExecutorDirs(Path[] dirs) { }; Path[] activeDirs = createLocalDirs(1); registerExecutor(testApp, prepareLocalDirs(activeDirs)); - String[] pushBlockIds = new String[] {"shuffle_0_0_0"}; + 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)); @@ -490,14 +466,25 @@ private void validateChunks( private void pushBlockHelper( String appId, - String[] blockIds, - ByteBuffer[] blocks) throws IOException { - Preconditions.checkArgument(blockIds.length == blocks.length); - for (int i = 0; i < blockIds.length; i++) { + 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, blockIds[i], 0)); - stream.onData(stream.getID(), blocks[i]); + 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; + PushBlock(int shuffleId, int mapIndex, int reduceId) { + this.shuffleId = shuffleId; + this.mapIndex = mapIndex; + this.reduceId = reduceId; + } + } } From 63843bb7bab021a02e681558a149d1df51d1d8ff Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Mon, 2 Nov 2020 12:05:57 -0800 Subject: [PATCH 28/35] Fixed typos, address review comments, made NoOp the default impl, and addressed the issue of race conditions --- .../spark/network/util/TransportConf.java | 11 +-- .../shuffle/RemoteBlockPushResolver.java | 80 ++++++++++--------- .../shuffle/protocol/PushBlockStream.java | 6 +- .../network/yarn/YarnShuffleService.java | 5 +- .../network/yarn/YarnShuffleServiceSuite.java | 29 +++---- 5 files changed, 70 insertions(+), 61 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index b1309d6ec9e1..9971aa0ce393 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -365,14 +365,15 @@ public boolean useOldFetchProtocol() { /** * 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 + * 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 mergeShuffleFileManagerImpl() { + public String mergedShuffleFileManagerImpl() { return conf.get("spark.shuffle.push.based.mergedShuffleFileManagerImpl", - "org.apache.spark.network.shuffle.RemoteBlockPushResolver"); + "org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager"); } /** diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index b2ed9164b7ed..a2bc24491e3b 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -115,6 +115,12 @@ public ShuffleIndexInformation load(File file) throws IOException { 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 shufflePartitions = partitions.computeIfAbsent(appShuffleId, id -> Maps.newConcurrentMap()); return shufflePartitions.computeIfAbsent(reduceId, key -> { @@ -122,7 +128,6 @@ private AppShufflePartitionInfo getOrCreateAppShufflePartitionInfo( // 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 { @@ -357,47 +362,50 @@ public void onFailure(String streamId, Throwable cause) { @Override public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) throws IOException { logger.info("Finalizing shuffle {} from Application {}.", msg.shuffleId, msg.appId); - Map shufflePartitions = - partitions.remove(new AppShuffleId(msg.appId, msg.shuffleId)); + AppShuffleId appShuffleId = new AppShuffleId(msg.appId, msg.shuffleId); + Map shufflePartitions = partitions.get(appShuffleId); + MergeStatuses mergeStatuses; 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 partitions = shufflePartitions.values(); - int totalPartitions = partitions.size(); - RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions]; - int[] reduceIds = new int[totalPartitions]; - long[] sizes = new long[totalPartitions]; - Iterator partitionsIter = partitions.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); + mergeStatuses = + new MergeStatuses(msg.shuffleId, new RoaringBitmap[0], new int[0], new long[0]); + } else { + Collection partitionsToFinalize = shufflePartitions.values(); + int totalPartitions = partitionsToFinalize.size(); + RoaringBitmap[] bitmaps = new RoaringBitmap[totalPartitions]; + int[] reduceIds = new int[totalPartitions]; + long[] sizes = new long[totalPartitions]; + Iterator 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(); } - 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 from shuffle maps 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 new MergeStatuses(msg.shuffleId, bitmaps, reduceIds, sizes); + return mergeStatuses; } @Override diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index 0521e47ceddb..ec997b004455 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -64,7 +64,7 @@ public int hashCode() { public String toString() { return Objects.toStringHelper(this) .add("appId", appId) - .add("streamId", shuffleId) + .add("shuffleId", shuffleId) .add("mapIndex", mapIndex) .add("reduceId", reduceId) .add("index", index) @@ -100,10 +100,10 @@ public void encode(ByteBuf buf) { public static PushBlockStream decode(ByteBuf buf) { String appId = Encoders.Strings.decode(buf); - int streamId = buf.readInt(); + int shuffleId = buf.readInt(); int mapIdx = buf.readInt(); int reduceId = buf.readInt(); int index = buf.readInt(); - return new PushBlockStream(appId, streamId, mapIdx, reduceId, index); + return new PushBlockStream(appId, shuffleId, mapIdx, reduceId, index); } } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java index 5b7f37f01d01..548a5ccd1385 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java @@ -42,7 +42,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.api.*; import org.apache.spark.network.shuffle.MergedShuffleFileManager; -import org.apache.spark.network.shuffle.RemoteBlockPushResolver; import org.apache.spark.network.util.LevelDBProvider; import org.iq80.leveldb.DB; import org.iq80.leveldb.DBIterator; @@ -226,7 +225,7 @@ protected void serviceInit(Configuration conf) throws Exception { @VisibleForTesting static MergedShuffleFileManager newMergedShuffleFileManagerInstance(TransportConf conf) { - String mergeManagerImplClassName = conf.mergeShuffleFileManagerImpl(); + String mergeManagerImplClassName = conf.mergedShuffleFileManagerImpl(); try { Class mergeManagerImplClazz = Class.forName( mergeManagerImplClassName, true, Thread.currentThread().getContextClassLoader()); @@ -237,7 +236,7 @@ static MergedShuffleFileManager newMergedShuffleFileManagerInstance(TransportCon return mergeManagerSubClazz.getConstructor(TransportConf.class).newInstance(conf); } catch (Exception e) { logger.error("Unable to create an instance of {}", mergeManagerImplClassName); - return new RemoteBlockPushResolver(conf); + return new ExternalBlockHandler.NoOpMergedShuffleFileManager(conf); } } diff --git a/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java b/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java index 0fb8f038b49d..09bc4d840367 100644 --- a/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java +++ b/common/network-yarn/src/test/java/org/apache/spark/network/yarn/YarnShuffleServiceSuite.java @@ -17,44 +17,45 @@ package org.apache.spark.network.yarn; -import org.apache.spark.network.shuffle.ExternalBlockHandler; -import org.apache.spark.network.shuffle.MergedShuffleFileManager; -import org.apache.spark.network.shuffle.RemoteBlockPushResolver; -import org.apache.spark.network.util.TransportConf; import org.junit.Test; import static org.junit.Assert.*; import static org.mockito.Mockito.*; +import org.apache.spark.network.shuffle.ExternalBlockHandler; +import org.apache.spark.network.shuffle.MergedShuffleFileManager; +import org.apache.spark.network.shuffle.RemoteBlockPushResolver; +import org.apache.spark.network.util.TransportConf; + public class YarnShuffleServiceSuite { @Test public void testCreateDefaultMergedShuffleFileManagerInstance() { TransportConf mockConf = mock(TransportConf.class); - when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( - "org.apache.spark.network.shuffle.RemoteBlockPushResolver"); + when(mockConf.mergedShuffleFileManagerImpl()).thenReturn( + "org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager"); MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( mockConf); - assertTrue(mergeMgr instanceof RemoteBlockPushResolver); + assertTrue(mergeMgr instanceof ExternalBlockHandler.NoOpMergedShuffleFileManager); } @Test - public void testCreateNoOpMergedShuffleFileManagerInstance() { + public void testCreateRemoteBlockPushResolverInstance() { TransportConf mockConf = mock(TransportConf.class); - when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( - "org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager"); + when(mockConf.mergedShuffleFileManagerImpl()).thenReturn( + "org.apache.spark.network.shuffle.RemoteBlockPushResolver"); MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( mockConf); - assertTrue(mergeMgr instanceof ExternalBlockHandler.NoOpMergedShuffleFileManager); + assertTrue(mergeMgr instanceof RemoteBlockPushResolver); } @Test - public void testInvalidClassNameOfMergeManagerWillUseRemoteBlockPushResolverImpl() { + public void testInvalidClassNameOfMergeManagerWillUseNoOpInstance() { TransportConf mockConf = mock(TransportConf.class); - when(mockConf.mergeShuffleFileManagerImpl()).thenReturn( + when(mockConf.mergedShuffleFileManagerImpl()).thenReturn( "org.apache.spark.network.shuffle.NotExistent"); MergedShuffleFileManager mergeMgr = YarnShuffleService.newMergedShuffleFileManagerInstance( mockConf); - assertTrue(mergeMgr instanceof RemoteBlockPushResolver); + assertTrue(mergeMgr instanceof ExternalBlockHandler.NoOpMergedShuffleFileManager); } } From d35aa4b75465b3e0c1e6ba75debb8e41a19bd738 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 3 Nov 2020 13:34:30 -0800 Subject: [PATCH 29/35] Addressed review comments --- .../spark/network/util/TransportConf.java | 2 +- .../network/shuffle/OneForOneBlockPusher.java | 4 +- .../shuffle/RemoteBlockPushResolver.java | 113 ++++++++---------- .../shuffle/protocol/PushBlockStream.java | 4 - .../shuffle/RemoteBlockPushResolverSuite.java | 59 ++++----- 5 files changed, 79 insertions(+), 103 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java index 9971aa0ce393..fd287b022618 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -372,7 +372,7 @@ public boolean useOldFetchProtocol() { * 'org.apache.spark.network.shuffle.RemoteBlockPushResolver'. */ public String mergedShuffleFileManagerImpl() { - return conf.get("spark.shuffle.push.based.mergedShuffleFileManagerImpl", + return conf.get("spark.shuffle.server.mergedShuffleFileManagerImpl", "org.apache.spark.network.shuffle.ExternalBlockHandler$NoOpMergedShuffleFileManager"); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java index 563f5dbec06e..9f0188039d6f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -39,6 +39,7 @@ public class OneForOneBlockPusher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockPusher.class); private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); + private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush"; private final TransportClient client; private final String appId; @@ -116,8 +117,7 @@ public void start() { assert buffers.containsKey(blockIds[i]) : "Could not find the block buffer for block " + blockIds[i]; String[] blockIdParts = blockIds[i].split("_"); - if (blockIdParts.length != 4 || !blockIdParts[0].equals( - PushBlockStream.SHUFFLE_PUSH_BLOCK_PREFIX)) { + if (blockIdParts.length != 4 || !blockIdParts[0].equals(SHUFFLE_PUSH_BLOCK_PREFIX)) { throw new IllegalArgumentException( "Unexpected shuffle push block id format: " + blockIds[i]); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index a2bc24491e3b..96b99f626db3 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -71,6 +71,7 @@ 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"; @@ -321,16 +322,17 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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, msg, appShuffleId, msg.reduceId, msg.mapIndex, partitionInfo); + 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 msg.streamId; + return streamId; } @Override @@ -344,7 +346,7 @@ 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, + 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 @@ -415,12 +417,8 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { + "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); - }); + appsPathInfo.putIfAbsent(appId, new AppPathsInfo(appId, executorInfo.localDirs, + executorInfo.subDirsPerLocalDir)); } private static String generateFileName(AppShuffleId appShuffleId, int reduceId) { @@ -434,9 +432,7 @@ private static String generateFileName(AppShuffleId appShuffleId, int reduceId) static class PushBlockStreamCallback implements StreamCallbackWithID { private final RemoteBlockPushResolver mergeManager; - private final PushBlockStream msg; - private final AppShuffleId appShuffleId; - private final int reduceId; + private final String streamId; private final int mapIndex; private final AppShufflePartitionInfo partitionInfo; private int length = 0; @@ -450,22 +446,18 @@ static class PushBlockStreamCallback implements StreamCallbackWithID { private PushBlockStreamCallback( RemoteBlockPushResolver mergeManager, - PushBlockStream msg, - AppShuffleId appShuffleId, - int reduceId, - int mapIndex, - AppShufflePartitionInfo partitionInfo) { + String streamId, + AppShufflePartitionInfo partitionInfo, + int mapIndex) { this.mergeManager = Preconditions.checkNotNull(mergeManager); - this.msg = Preconditions.checkNotNull(msg); - this.appShuffleId = appShuffleId; - this.reduceId = reduceId; - this.mapIndex = mapIndex; + this.streamId = streamId; this.partitionInfo = Preconditions.checkNotNull(partitionInfo); + this.mapIndex = mapIndex; } @Override public String getID() { - return msg.streamId; + return streamId; } /** @@ -480,8 +472,8 @@ private void writeBuf(ByteBuffer buf) throws IOException { long updatedPos = partitionInfo.getPosition() + length; logger.debug( "{} shuffleId {} reduceId {} encountered failure current pos {} updated pos {}", - appShuffleId.appId, appShuffleId.shuffleId, reduceId, partitionInfo.getPosition(), - updatedPos); + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId, partitionInfo.getPosition(), updatedPos); length += partitionInfo.dataChannel.write(buf, updatedPos); } else { length += partitionInfo.dataChannel.write(buf); @@ -548,13 +540,13 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { // memory, while still providing the necessary guarantee. synchronized (partitionInfo) { Map shufflePartitions = - mergeManager.partitions.get(appShuffleId); + 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(reduceId)) { + if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) { // TODO is it necessary to dereference deferredBufs? deferredBufs = null; return; @@ -568,8 +560,9 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { deferredBufs = null; return; } - logger.trace("{} shuffleId {} reduceId {} onData writable", appShuffleId.appId, - appShuffleId.shuffleId, reduceId); + logger.trace("{} shuffleId {} reduceId {} onData writable", + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId); if (partitionInfo.getCurrentMapIndex() < 0) { partitionInfo.setCurrentMapIndex(mapIndex); } @@ -589,8 +582,9 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { partitionInfo.setEncounteredFailure(false); } } else { - logger.trace("{} shuffleId {} reduceId {} onData deferred", appShuffleId.appId, - appShuffleId.shuffleId, reduceId); + 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 @@ -628,16 +622,17 @@ public void onComplete(String streamId) throws IOException { // 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); + logger.trace("{} shuffleId {} reduceId {} onComplete invoked", + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId); Map shufflePartitions = - mergeManager.partitions.get(appShuffleId); + 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(reduceId)) { + if (shufflePartitions == null || !shufflePartitions.containsKey(partitionInfo.reduceId)) { deferredBufs = null; - throw new RuntimeException(String.format("Block %s %s", msg.streamId, + throw new RuntimeException(String.format("Block %s %s", streamId, ErrorHandler.BlockPushErrorHandler.TOO_LATE_MESSAGE_SUFFIX)); } // Check if we can commit this block @@ -670,7 +665,7 @@ public void onComplete(String streamId) throws IOException { deferredBufs = null; throw new RuntimeException(String.format("%s %s to merged shuffle", ErrorHandler.BlockPushErrorHandler.BLOCK_APPEND_COLLISION_DETECTED_MSG_PREFIX, - msg.streamId)); + streamId)); } } isWriting = false; @@ -679,9 +674,9 @@ public void onComplete(String streamId) throws IOException { @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); + logger.error("Encountered issue when merging {}", streamId, throwable); } else { - logger.debug("Encountered issue when merging shuffle partition block {}", msg, throwable); + 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 @@ -690,10 +685,11 @@ public void onFailure(String streamId, Throwable throwable) throws IOException { if (isWriting) { synchronized (partitionInfo) { Map shufflePartitions = - mergeManager.partitions.get(appShuffleId); - if (shufflePartitions != null && shufflePartitions.containsKey(reduceId)) { + mergeManager.partitions.get(partitionInfo.appShuffleId); + if (shufflePartitions != null && shufflePartitions.containsKey(partitionInfo.reduceId)) { logger.debug("{} shuffleId {} reduceId {} set encountered failure", - appShuffleId.appId, appShuffleId.shuffleId, reduceId); + partitionInfo.appShuffleId.appId, partitionInfo.appShuffleId.shuffleId, + partitionInfo.reduceId); partitionInfo.setCurrentMapIndex(-1); partitionInfo.setEncounteredFailure(true); } @@ -938,29 +934,26 @@ protected void finalize() throws Throwable { */ private static class AppPathsInfo { - private String[] activeLocalDirs; - private int subDirsPerLocalDir; + private final String[] activeLocalDirs; + private final int subDirsPerLocalDir; - private AppPathsInfo updateActiveLocalDirs( + private AppPathsInfo( 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); - } + 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 active local dirs {} and sub dirs {} for application {}", + Arrays.toString(activeLocalDirs),subDirsPerLocalDir, appId); } - return this; } } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java index ec997b004455..5d14091f5c40 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/PushBlockStream.java @@ -30,7 +30,6 @@ * 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 int shuffleId; public final int mapIndex; @@ -38,7 +37,6 @@ public class PushBlockStream extends BlockTransferMessage { // 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; public PushBlockStream(String appId, int shuffleId, int mapIndex, int reduceId, int index) { this.appId = appId; @@ -46,8 +44,6 @@ public PushBlockStream(String appId, int shuffleId, int mapIndex, int reduceId, this.mapIndex = mapIndex; this.reduceId = reduceId; this.index = index; - this.streamId = String.format("%s_%d_%d_%d", SHUFFLE_PUSH_BLOCK_PREFIX, - shuffleId, mapIndex, reduceId); } @Override diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 79f2957b0cc1..2df31d4e826e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -26,7 +26,6 @@ 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; @@ -98,12 +97,11 @@ public void testNoIndexFile() { @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]) + PushBlock[] pushBlocks = new PushBlock[] { + new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4])), + new PushBlock(0, 1, 0, ByteBuffer.wrap(new byte[5])) }; - pushBlockHelper(TEST_APP, pushBlocks, blocks); + pushBlockHelper(TEST_APP, pushBlocks); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0}, new long[] {9}); @@ -114,18 +112,12 @@ public void testBasicBlockMerge() throws IOException { @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]) + new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[2])), + new PushBlock(0, 1, 0, ByteBuffer.wrap(new byte[3])), + new PushBlock(0, 2, 0, ByteBuffer.wrap(new byte[5])), + new PushBlock(0, 3, 0, ByteBuffer.wrap(new byte[3])) }; - pushBlockHelper(TEST_APP, pushBlocks, buffers); + pushBlockHelper(TEST_APP, pushBlocks); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0}, new long[] {13}); @@ -136,17 +128,12 @@ public void testDividingMergedBlocksIntoChunks() throws IOException { @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]) + new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[2])), + new PushBlock(0, 1, 0, ByteBuffer.wrap(new byte[3])), + new PushBlock(0, 0, 1, ByteBuffer.wrap(new byte[5])), + new PushBlock(0, 1, 1, ByteBuffer.wrap(new byte[3])) }; - pushBlockHelper(TEST_APP, pushBlocks, buffers); + pushBlockHelper(TEST_APP, pushBlocks); MergeStatuses statuses = pushResolver.finalizeShuffleMerge( new FinalizeShuffleMerge(TEST_APP, 0)); validateMergeStatuses(statuses, new int[] {0, 1}, new long[] {5, 8}); @@ -332,7 +319,7 @@ public void testFailureWith3Streams() throws IOException { // Since this stream didn't get any opportunity it will throw couldn't find opportunity error RuntimeException failedEx = null; try { - stream3.onComplete(stream2.getID()); + stream3.onComplete(stream3.getID()); } catch (RuntimeException re) { assertEquals( "Couldn't find an opportunity to write block shufflePush_0_2_0 to merged shuffle", @@ -389,9 +376,9 @@ void deleteExecutorDirs(Path[] dirs) { }; 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); + PushBlock[] pushBlocks = new PushBlock[] { + new PushBlock(0, 0, 0, ByteBuffer.wrap(new byte[4]))}; + pushBlockHelper(testApp, pushBlocks); pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, 0)); MergedBlockMeta blockMeta = pushResolver.getMergedBlockMeta(testApp, 0, 0); validateChunks(testApp, 0, 0, blockMeta, new int[]{4}, new int[][]{{0}}); @@ -466,13 +453,11 @@ private void validateChunks( private void pushBlockHelper( String appId, - PushBlock[] blocks, - ByteBuffer[] buffers) throws IOException { - Preconditions.checkArgument(blocks.length == buffers.length); + PushBlock[] blocks) throws IOException { 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.onData(stream.getID(), blocks[i].buffer); stream.onComplete(stream.getID()); } } @@ -481,10 +466,12 @@ private static class PushBlock { private final int shuffleId; private final int mapIndex; private final int reduceId; - PushBlock(int shuffleId, int mapIndex, int reduceId) { + private final ByteBuffer buffer; + PushBlock(int shuffleId, int mapIndex, int reduceId, ByteBuffer buffer) { this.shuffleId = shuffleId; this.mapIndex = mapIndex; this.reduceId = reduceId; + this.buffer = buffer; } } } From ba9231140760df6f3394dfaaeb0217bd7539d6ce Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Tue, 3 Nov 2020 23:35:56 -0800 Subject: [PATCH 30/35] Fix IndexOutOfBoundsException and avoid instantiating AppsPathInfo multiple times --- .../main/java/org/apache/spark/network/protocol/Encoders.java | 1 + .../apache/spark/network/shuffle/RemoteBlockPushResolver.java | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index 4fa191b3917e..d0d5d23caa8e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -62,6 +62,7 @@ public static void encode(ByteBuf buf, RoaringBitmap b) { // 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. + buf.ensureWritable(encodedLength); b.serialize(buf.nioBuffer(buf.writerIndex(), encodedLength)); buf.writerIndex(buf.writerIndex() + encodedLength); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index 96b99f626db3..d5a16ddc0540 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -417,7 +417,7 @@ public void registerExecutor(String appId, ExecutorShuffleInfo executorInfo) { + "num sub-dirs {}", appId, Arrays.toString(executorInfo.localDirs), executorInfo.subDirsPerLocalDir); } - appsPathInfo.putIfAbsent(appId, new AppPathsInfo(appId, executorInfo.localDirs, + appsPathInfo.computeIfAbsent(appId, id -> new AppPathsInfo(appId, executorInfo.localDirs, executorInfo.subDirsPerLocalDir)); } From 9be25b39272bfaaff64662ea3c79ff5820c29509 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 4 Nov 2020 09:08:42 -0800 Subject: [PATCH 31/35] Removed duplicate declaration of shuffle push prefix --- .../apache/spark/network/shuffle/OneForOneBlockPusher.java | 2 +- .../spark/network/shuffle/RemoteBlockPushResolver.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java index 9f0188039d6f..1bde1edcfaff 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockPusher.java @@ -39,7 +39,7 @@ public class OneForOneBlockPusher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockPusher.class); private static final ErrorHandler PUSH_ERROR_HANDLER = new ErrorHandler.BlockPushErrorHandler(); - private static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush"; + public static final String SHUFFLE_PUSH_BLOCK_PREFIX = "shufflePush"; private final TransportClient client; private final String appId; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java index d5a16ddc0540..9391ed6dcb62 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java @@ -71,7 +71,6 @@ 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"; @@ -322,8 +321,9 @@ public StreamCallbackWithID receiveBlockDataAsStream(PushBlockStream msg) { 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); + 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 { From ba5179624fc495237a15bd12c3221df9e6894d6b Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 4 Nov 2020 09:24:47 -0800 Subject: [PATCH 32/35] Added UT for collision with 2 streams --- .../shuffle/RemoteBlockPushResolverSuite.java | 23 +++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java index 2df31d4e826e..0f200dc72196 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/RemoteBlockPushResolverSuite.java @@ -303,11 +303,30 @@ public void testIncompleteStreamsAreOverwritten() throws IOException { } @Test (expected = RuntimeException.class) - public void testFailureWith3Streams() throws IOException { + public void testCollision() 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[5])); + // Since stream2 didn't get any opportunity it will throw couldn't find opportunity error + try { + stream2.onComplete(stream2.getID()); + } catch (RuntimeException re) { + assertEquals( + "Couldn't find an opportunity to write block shufflePush_0_1_0 to merged shuffle", + re.getMessage()); + throw re; + } + } + + @Test (expected = RuntimeException.class) + public void testFailureInAStreamDoesNotInterfereWithStreamWhichIsWriting() 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 From 1f4fcfec6643b6c382287b91f4744a67fb408018 Mon Sep 17 00:00:00 2001 From: Chandni Singh Date: Wed, 4 Nov 2020 14:42:05 -0800 Subject: [PATCH 33/35] Removed unnecessary TODOs, marked MergedShuffleFileManager evolving, added since tag to all the new classes introduced --- .../spark/network/protocol/Encoders.java | 6 +- common/network-shuffle/pom.xml | 10 ++-- .../spark/network/shuffle/ErrorHandler.java | 8 ++- .../network/shuffle/ExternalBlockHandler.java | 2 + .../network/shuffle/MergedBlockMeta.java | 2 + .../shuffle/MergedShuffleFileManager.java | 5 +- .../network/shuffle/OneForOneBlockPusher.java | 2 + .../shuffle/RemoteBlockPushResolver.java | 56 +++++++++---------- .../protocol/FinalizeShuffleMerge.java | 2 + .../shuffle/protocol/MergeStatuses.java | 2 + .../shuffle/protocol/PushBlockStream.java | 3 +- 11 files changed, 61 insertions(+), 37 deletions(-) diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index d0d5d23caa8e..1922935d9c2e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -173,7 +173,11 @@ public static long[] decode(ByteBuf buf) { } } - /** Bitmap arrays are encoded with the number of bitmaps followed by per-Bitmap encoding. */ + /** + * Bitmap arrays are encoded with the number of bitmaps followed by per-Bitmap encoding. + * + * @since 3.1.0 + */ public static class BitmapArrays { public static int encodedLength(RoaringBitmap[] bitmaps) { int totalLength = 4; diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index a4a1ff92ef9a..562a1d495cc8 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -47,6 +47,11 @@ metrics-core + + org.apache.spark + spark-tags_${scala.binary.version} + + org.slf4j @@ -70,11 +75,6 @@ test-jar test - - org.apache.spark - spark-tags_${scala.binary.version} - test -